From f3916c57b1d9f6b987dead43eaf9800da4c03a38 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 17 Aug 2023 15:55:01 -0700 Subject: [PATCH 01/42] initial impl --- .../SnowflakeSinkConnectorConfig.java | 2 +- .../kafka/connector/SnowflakeSinkTask.java | 1 - .../connector/internal/SnowflakeErrors.java | 3 +- .../internal/SnowflakeSinkService.java | 6 - .../internal/SnowflakeSinkServiceFactory.java | 27 +--- .../internal/SnowflakeSinkServiceV1.java | 19 +-- .../streaming/SnowflakeSinkServiceV2.java | 19 +-- .../streaming/TopicPartitionChannel.java | 5 +- .../connector/records/RecordService.java | 88 +++++------ .../connector/internal/MetaColumnIT.java | 3 +- .../connector/internal/SinkServiceIT.java | 146 +++++++++++++----- ...SnowflakeTelemetryPipeStatusMetricsIT.java | 5 +- .../streaming/SnowflakeSinkServiceV2IT.java | 38 +++++ .../streaming/TopicPartitionChannelTest.java | 4 +- .../connector/records/RecordContentTest.java | 6 +- 15 files changed, 208 insertions(+), 164 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java index 4adbc121f..d7b86c39a 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java @@ -678,7 +678,7 @@ public static String[] names() { @Override public String toString() { - return name().toLowerCase(Locale.ROOT); + return name(); } } diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java index 335aebd22..3d27734e2 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java @@ -221,7 +221,6 @@ public void start(final Map parsedConfig) { .setFlushTime(bufferFlushTime) .setTopic2TableMap(topic2table) .setMetadataConfig(metadataConfig) - .setBehaviorOnNullValuesConfig(behavior) .setCustomJMXMetrics(enableCustomJMXMonitoring) .setErrorReporter(kafkaRecordErrorReporter) .setSinkTaskContext(this.context) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index 9dc4424f4..a30cc54ac 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -17,6 +17,7 @@ package com.snowflake.kafka.connector.internal; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; @@ -261,7 +262,7 @@ public enum SnowflakeErrors { ERROR_5016( "5016", "Invalid SinkRecord received", - "SinkRecord.value and SinkRecord.valueSchema cannot be null"), + "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion is enabled (see " + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + " for more information."), ERROR_5017( "5017", "Invalid api call to cached put", "Cached put only support AWS, Azure and GCS."), ERROR_5018("5018", "Failed to execute cached put", "Error in cached put command"), diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java index 6983d0572..4a025a6f4 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java @@ -134,15 +134,9 @@ public interface SnowflakeSinkService { /** @return current file size limitation */ long getFileSize(); - /* Set the behavior on what action to perform when this( @see com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig#BEHAVIOR_ON_NULL_VALUES_CONFIG ) config is set. */ - void setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior); - /* Should we emit Custom SF JMX Metrics to Mbean Server? If true (Default), we emit in form of SimpleMbeans */ void setCustomJMXMetrics(boolean enableJMX); - /* Only used in testing and verifying what was the passed value of this behavior from config to sink service*/ - SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig(); - /* Set Error reporter which can be used to send records to DLQ (Dead Letter Queue) */ default void setErrorReporter(KafkaRecordErrorReporter kafkaRecordErrorReporter) {} diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java index e0eacf173..fad9bac59 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java @@ -5,6 +5,8 @@ import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; + +import java.util.HashMap; import java.util.Map; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -23,17 +25,7 @@ public static SnowflakeSinkServiceBuilder builder( SnowflakeConnectionService conn, IngestionMethodConfig ingestionType, Map connectorConfig) { - return new SnowflakeSinkServiceBuilder(conn, ingestionType, connectorConfig); - } - - /** - * Basic builder which internally uses SinkServiceV1 (Snowpipe) - * - * @param conn connection instance for connecting to snowflake - * @return A wrapper(Builder) having SinkService instance - */ - public static SnowflakeSinkServiceBuilder builder(SnowflakeConnectionService conn) { - return new SnowflakeSinkServiceBuilder(conn); + return new SnowflakeSinkServiceBuilder(conn, ingestionType, connectorConfig == null ? new HashMap<>() : connectorConfig); } /** Builder class to create instance of {@link SnowflakeSinkService} */ @@ -47,7 +39,7 @@ private SnowflakeSinkServiceBuilder( IngestionMethodConfig ingestionType, Map connectorConfig) { if (ingestionType == IngestionMethodConfig.SNOWPIPE) { - this.service = new SnowflakeSinkServiceV1(conn); + this.service = new SnowflakeSinkServiceV1(conn, connectorConfig); } else { this.service = new SnowflakeSinkServiceV2(conn, connectorConfig); } @@ -55,10 +47,6 @@ private SnowflakeSinkServiceBuilder( LOGGER.info("{} created", this.service.getClass().getName()); } - private SnowflakeSinkServiceBuilder(SnowflakeConnectionService conn) { - this(conn, IngestionMethodConfig.SNOWPIPE, null /* Not required for V1 */); - } - /** * Add task for table and TopicPartition. Mostly used only for testing. When connector starts, * startTask is directly called. @@ -111,13 +99,6 @@ public SnowflakeSinkServiceBuilder setMetadataConfig(SnowflakeMetadataConfig con return this; } - public SnowflakeSinkServiceBuilder setBehaviorOnNullValuesConfig( - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { - this.service.setBehaviorOnNullValuesConfig(behavior); - LOGGER.info("Config Behavior on null value is {}", behavior.toString()); - return this; - } - public SnowflakeSinkServiceBuilder setCustomJMXMetrics(final boolean enableJMX) { this.service.setCustomJMXMetrics(enableJMX); LOGGER.info("Config JMX value {}. (true = Enabled, false = Disabled)", enableJMX); diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index 530e8851a..9e1130f08 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -76,7 +76,7 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { // If this is true, we will enable Mbean for required classes and emit JMX metrics for monitoring private boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; - SnowflakeSinkServiceV1(SnowflakeConnectionService conn) { + SnowflakeSinkServiceV1(SnowflakeConnectionService conn, Map connectorConfig) { if (conn == null || conn.isClosed()) { throw SnowflakeErrors.ERROR_5010.getException(); } @@ -88,12 +88,14 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { this.conn = conn; isStopped = false; this.telemetryService = conn.getTelemetryClient(); - this.recordService = new RecordService(this.telemetryService); + this.recordService = new RecordService(connectorConfig); this.topic2TableMap = new HashMap<>(); // Setting the default value in constructor // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; + this.behaviorOnNullValues = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( + connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); } /** @@ -303,22 +305,11 @@ public long getFileSize() { return this.fileSize; } - @Override - public void setBehaviorOnNullValuesConfig( - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { - this.behaviorOnNullValues = behavior; - } - @Override public void setCustomJMXMetrics(boolean enableJMX) { this.enableCustomJMXMonitoring = enableJMX; } - @Override - public SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig() { - return this.behaviorOnNullValues; - } - /** * Loop through all pipes in memory and find out the metric registry instance for that pipe. The * pipes object's key is not pipeName hence need to loop over. diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 884a91d29..142784c4b 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -109,7 +109,7 @@ public SnowflakeSinkServiceV2( this.flushTimeSeconds = StreamingUtils.STREAMING_BUFFER_FLUSH_TIME_DEFAULT_SEC; this.conn = conn; this.telemetryService = conn.getTelemetryClient(); - this.recordService = new RecordService(this.telemetryService); + this.recordService = new RecordService(connectorConfig); this.topicToTableMap = new HashMap<>(); // Setting the default value in constructor @@ -118,8 +118,10 @@ public SnowflakeSinkServiceV2( this.connectorConfig = connectorConfig; - this.enableSchematization = - this.recordService.setAndGetEnableSchematizationFromConfig(this.connectorConfig); + this.enableSchematization = Boolean.parseBoolean(connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + this.behaviorOnNullValues = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( + connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() @@ -463,22 +465,11 @@ public long getFileSize() { return this.fileSizeBytes; } - @Override - public void setBehaviorOnNullValuesConfig( - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { - this.behaviorOnNullValues = behavior; - } - @Override public void setCustomJMXMetrics(boolean enableJMX) { this.enableCustomJMXMonitoring = enableJMX; } - @Override - public SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig() { - return this.behaviorOnNullValues; - } - /* Set this to send records to DLQ. */ @Override public void setErrorReporter(KafkaRecordErrorReporter kafkaRecordErrorReporter) { diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java index dbf0b8f22..ebb939924 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java @@ -11,6 +11,7 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.internal.BufferThreshold; @@ -258,9 +259,7 @@ public TopicPartitionChannel( !Strings.isNullOrEmpty(StreamingUtils.getDlqTopicName(this.sfConnectorConfig)); /* Schematization related properties */ - this.enableSchematization = - this.recordService.setAndGetEnableSchematizationFromConfig(sfConnectorConfig); - + this.enableSchematization = Boolean.parseBoolean(sfConnectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); this.enableSchemaEvolution = this.enableSchematization && hasSchemaEvolutionPermission; // Open channel and reset the offset in kafka diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 0bccd8d8a..ee50e8aae 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -23,7 +23,7 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.internal.KCLogger; import com.snowflake.kafka.connector.internal.SnowflakeErrors; -import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.text.SimpleDateFormat; @@ -70,8 +70,6 @@ public class RecordService { private static final String KEY_SCHEMA_ID = "key_schema_id"; static final String HEADERS = "headers"; - private boolean enableSchematization = false; - // For each task, we require a separate instance of SimpleDataFormat, since they are not // inherently thread safe static final ThreadLocal ISO_DATE_TIME_FORMAT = @@ -89,60 +87,44 @@ public class RecordService { // This class is designed to work with empty metadata config map private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); - - /** Send Telemetry Data to Snowflake */ - private final SnowflakeTelemetryService telemetryService; + private Map connectorConfig; + private boolean enableSchematization; + private boolean ingestTombstoneRecords; /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", * "partition": 123, "key":"key name" } "content": "record content" } * *

create a JsonRecordService instance - * - * @param telemetryService Telemetry Service Instance. Can be null. */ - public RecordService(SnowflakeTelemetryService telemetryService) { - this.telemetryService = telemetryService; - } + public RecordService(Map connectorConfig) { + this.connectorConfig = connectorConfig; - /** Record service with null telemetry Service, only use it for testing. */ - @VisibleForTesting - public RecordService() { - this.telemetryService = null; - } - - public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { - metadataConfig = metadataConfigIn; - } - - /** - * extract enableSchematization from the connector config and set the value for the recordService - * - *

The extracted boolean is returned for external usage. - * - * @param connectorConfig the connector config map - * @return a boolean indicating whether schematization is enabled - */ - public boolean setAndGetEnableSchematizationFromConfig( - final Map connectorConfig) { if (connectorConfig.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) { this.enableSchematization = Boolean.parseBoolean( connectorConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)); } - return this.enableSchematization; + if (connectorConfig.containsKey(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)) { + this.ingestTombstoneRecords = connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG).equalsIgnoreCase(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + } } - /** - * Directly set the enableSchematization through param - * - *

This method is only for testing - * - * @param enableSchematization whether we should enable schematization or not - */ + // TESTING ONLY - inject schematization and tombstone behavior @VisibleForTesting - public void setEnableSchematization(final boolean enableSchematization) { + public RecordService(boolean enableSchematization, boolean ingestTombstoneRecords) { this.enableSchematization = enableSchematization; + this.ingestTombstoneRecords = ingestTombstoneRecords; + } + + // TESTING ONLY - create empty record service + @VisibleForTesting + public RecordService() { + + } + + public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { + metadataConfig = metadataConfigIn; } /** @@ -152,19 +134,25 @@ public void setEnableSchematization(final boolean enableSchematization) { * @return a Row wrapper which contains both actual content(payload) and metadata */ private SnowflakeTableRow processRecord(SinkRecord record) { + SnowflakeRecordContent valueContent; + if (record.value() == null || record.valueSchema() == null) { - throw SnowflakeErrors.ERROR_5016.getException(); - } - if (!record.valueSchema().name().equals(SnowflakeJsonSchema.NAME)) { - throw SnowflakeErrors.ERROR_0009.getException(); - } - if (!(record.value() instanceof SnowflakeRecordContent)) { - throw SnowflakeErrors.ERROR_0010.getException( - "Input record should be SnowflakeRecordContent object"); + if (this.ingestTombstoneRecords) { + valueContent = new SnowflakeRecordContent(); + } else { + throw SnowflakeErrors.ERROR_5016.getException(); + } + } else { + if (!record.valueSchema().name().equals(SnowflakeJsonSchema.NAME)) { + throw SnowflakeErrors.ERROR_0009.getException(); + } + if (!(record.value() instanceof SnowflakeRecordContent)) { + throw SnowflakeErrors.ERROR_0010.getException( + "Input record should be SnowflakeRecordContent object"); + } + valueContent = (SnowflakeRecordContent) record.value(); } - SnowflakeRecordContent valueContent = (SnowflakeRecordContent) record.value(); - ObjectNode meta = MAPPER.createObjectNode(); if (metadataConfig.topicFlag) { meta.put(TOPIC, record.topic()); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java b/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java index bbb0a3478..66b708937 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java @@ -1,6 +1,7 @@ package com.snowflake.kafka.connector.internal; import com.snowflake.kafka.connector.Utils; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import java.nio.charset.StandardCharsets; @@ -36,7 +37,7 @@ public void testKey() throws Exception { conn.createStage(stageName); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(3) .build(); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index eec6864e6..810c91b96 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -6,6 +6,7 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import io.confluent.connect.avro.AvroConverter; @@ -17,6 +18,7 @@ import java.util.*; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.common.Configurable; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -51,7 +53,7 @@ public void afterEach() { @Test public void testSinkServiceBuilder() { // default value - SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn).build(); + SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); assert service.getFileSize() == SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT; assert service.getFlushTime() == SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_DEFAULT; @@ -59,7 +61,7 @@ public void testSinkServiceBuilder() { // set some value service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setFileSize(SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT * 4) .setFlushTime(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN + 10) .setRecordNumber(10) @@ -71,7 +73,7 @@ public void testSinkServiceBuilder() { // set some invalid value service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(-100) .setFlushTime(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN - 10) .setFileSize(SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MIN - 1) @@ -83,13 +85,13 @@ public void testSinkServiceBuilder() { // connection test assert TestUtils.assertError( - SnowflakeErrors.ERROR_5010, () -> SnowflakeSinkServiceFactory.builder(null).build()); + SnowflakeErrors.ERROR_5010, () -> SnowflakeSinkServiceFactory.builder(null, IngestionMethodConfig.SNOWPIPE, null).build()); assert TestUtils.assertError( SnowflakeErrors.ERROR_5010, () -> { SnowflakeConnectionService conn = TestUtils.getConnectionService(); conn.close(); - SnowflakeSinkServiceFactory.builder(conn).build(); + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); }); } @@ -98,7 +100,7 @@ public void testIngestion() throws Exception { conn.createTable(table); conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -150,18 +152,16 @@ public void testIngestion() throws Exception { public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception { conn.createTable(table); conn.createStage(stage); + + Map connectorConfig = new HashMap<>(); + connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); - // Verifying it here to see if it fallbacks to default behavior - which is to ingest empty json - // string - Assert.assertTrue( - service - .getBehaviorOnNullValuesConfig() - .equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT)); SnowflakeConverter converter = new SnowflakeJsonConverter(); SchemaAndValue input = converter.toConnectData(topic, null); long offset = 0; @@ -217,19 +217,16 @@ public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() th public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception { conn.createTable(table); conn.createStage(stage); + + Map connectorConfig = new HashMap<>(); + connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); + SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) - .setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE) .build(); - // Verifying it here to see if it fallbacks to default behavior - which is to ingest empty json - // string - Assert.assertTrue( - service - .getBehaviorOnNullValuesConfig() - .equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); SnowflakeConverter converter = new SnowflakeJsonConverter(); SchemaAndValue input = converter.toConnectData(topic, null); long offset = 0; @@ -261,24 +258,89 @@ public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() thr service.closeAll(); } + @Test + public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() throws Exception { + conn.createTable(table); + conn.createStage(stage); + + Map connectorConfig = new HashMap<>(); + connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) + .setRecordNumber(1) + .addTask(table, new TopicPartition(topic, partition)) + .build(); + + JsonConverter converter = new JsonConverter(); + HashMap converterConfig = new HashMap(); + converterConfig.put("schemas.enable", "false"); + converter.configure(converterConfig, false); + SchemaAndValue input = converter.toConnectData(topic, null); + long offset = 0; + + SinkRecord record1 = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); + service.insert(Collections.singletonList(record1)); + TestUtils.assertWithRetry( + () -> + conn.listStage( + stage, + FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) + .size() + == 1, + 5, + 4); + service.callAllGetOffset(); + List files = + conn.listStage( + stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)); + String fileName = files.get(0); + + assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); + assert FileNameUtils.fileNameToPartition(fileName) == partition; + assert FileNameUtils.fileNameToStartOffset(fileName) == offset; + assert FileNameUtils.fileNameToEndOffset(fileName) == offset; + + // wait for ingest + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + + ResultSet resultSet = TestUtils.showTable(table); + LinkedList contentResult = new LinkedList<>(); + while (resultSet.next()) { + contentResult.add(resultSet.getString("RECORD_CONTENT")); + } + resultSet.close(); + + assert contentResult.size() == 1; + + ObjectNode emptyNode = MAPPER.createObjectNode(); + assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); + + // change cleaner + TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition) == 0, 30, 20); + + assert service.getOffset(new TopicPartition(topic, partition)) == offset + 1; + + service.closeAll(); + } + @Test public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() throws Exception { conn.createTable(table); conn.createStage(stage); + + Map connectorConfig = new HashMap<>(); + connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) - .setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE) .build(); - // Verifying it here to see if it fallbacks to default behavior - which is to ingest empty json - // string - Assert.assertTrue( - service - .getBehaviorOnNullValuesConfig() - .equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); JsonConverter converter = new JsonConverter(); HashMap converterConfig = new HashMap(); converterConfig.put("schemas.enable", "false"); @@ -393,7 +455,7 @@ public void testNativeJsonInputIngestion() throws Exception { startOffset + 3); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -581,7 +643,7 @@ public void testNativeAvroInputIngestion() throws Exception { conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -664,7 +726,7 @@ public void testNativeBrokenIngestion() throws Exception { startOffset + 2); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -704,7 +766,7 @@ public void testNativeNullIngestion() throws Exception { topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 2); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -756,7 +818,7 @@ public void testNativeNullValueIngestion() throws Exception { SinkRecord brokenValue = new SinkRecord(topic, partition, null, null, null, null, 0); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -773,7 +835,7 @@ public void testRecordNumber() throws Exception { int numLimit = 100; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(numLimit) .setFlushTime(30) .addTask(table, new TopicPartition(topic, partition)) @@ -820,7 +882,7 @@ public void testFileSize() throws Exception { long size = 10000; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setFileSize(size) .setFlushTime(10) .addTask(table, new TopicPartition(topic, partition)) @@ -852,7 +914,7 @@ public void testFlushTime() throws Exception { long flushTime = 20; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setFlushTime(flushTime) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -877,7 +939,7 @@ public void testSinkServiceNegative() { conn.createTable(table); conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn).setRecordNumber(1).build(); + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).setRecordNumber(1).build(); TopicPartition topicPartition = new TopicPartition(topic, partition); service.getOffset(topicPartition); List topicPartitionList = new ArrayList<>(); @@ -925,7 +987,7 @@ public void testRecoverReprocessFiles() throws Exception { assert getStageSize(stage, table, 0) == 4; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .addTask(table, new TopicPartition(topic, partition)) .setRecordNumber(1) // immediate flush .build(); @@ -974,7 +1036,7 @@ public void testBrokenRecord() { offset); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -1007,7 +1069,7 @@ public void testCleanerRecover() throws Exception { SnowflakeConnectionService spyConn = spy(conn); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(spyConn) + SnowflakeSinkServiceFactory.builder(spyConn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -1074,7 +1136,7 @@ public void testCleanerRecoverListCount() throws Exception { SnowflakeConnectionService spyConn = spy(conn); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(spyConn) + SnowflakeSinkServiceFactory.builder(spyConn, IngestionMethodConfig.SNOWPIPE, null) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java index 69fc4fa30..ce49639e3 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java @@ -6,6 +6,7 @@ import com.codahale.metrics.MetricRegistry; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.internal.metrics.MetricsUtil; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import java.nio.charset.StandardCharsets; @@ -43,7 +44,7 @@ public void testJMXMetricsInMBeanServer() throws Exception { // This means that default is true. SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(1) .build(); @@ -181,7 +182,7 @@ public void testJMXDisabledInMBeanServer() { // This means that default is true. SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn) + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) .setCustomJMXMetrics(false) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(1) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 2aaf85159..868b587cc 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -1357,6 +1357,44 @@ public void testStreamingIngestion_invalid_file_version() throws Exception { } } + @Test + public void testTombstoneIngestion() throws Exception { + Map config = TestUtils.getConfForStreaming(); + SnowflakeSinkConnectorConfig.setDefaultValues(config); + conn.createTable(table); + + // opens a channel for partition 0, table and topic + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config) + .setRecordNumber(1) + .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) + .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) + .addTask(table, new TopicPartition(topic, partition)) // Internally calls startTask + .build(); + + SnowflakeConverter converter = new SnowflakeJsonConverter(); + SchemaAndValue input = + converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); + long offset = 0; + + SinkRecord record1 = + new SinkRecord( + topic, + partition, + Schema.STRING_SCHEMA, + "test_key" + offset, + input.schema(), + null, + offset); + + service.insert(record1); + + TestUtils.assertWithRetry( + () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); + + service.closeAll(); + } + private void createNonNullableColumn(String tableName, String colName) { String createTableQuery = "alter table identifier(?) add " + colName + " int not null"; diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java index 1c20b0798..322d9e46a 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java @@ -224,7 +224,7 @@ public void testCloseChannelException() throws Exception { mockKafkaRecordErrorReporter, mockSinkTaskContext, mockSnowflakeConnectionService, - new RecordService(mockTelemetryService), + new RecordService(), mockTelemetryService); topicPartitionChannel.closeChannel(); @@ -614,7 +614,7 @@ public void testInsertRows_ValidationResponseHasErrors_NoErrorTolerance() throws mockKafkaRecordErrorReporter, mockSinkTaskContext, mockSnowflakeConnectionService, - new RecordService(mockTelemetryService), + new RecordService(), mockTelemetryService); List records = TestUtils.createJsonStringSinkRecords(0, 1, TOPIC, PARTITION); diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 0037d7060..04ac21006 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -235,10 +235,9 @@ public void testConvertToJsonReadOnlyByteBuffer() { @Test public void testSchematizationStringField() throws JsonProcessingException { - RecordService service = new RecordService(); + RecordService service = new RecordService(true, false); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); - service.setEnableSchematization(true); String value = "{\"name\":\"sf\",\"answer\":42}"; byte[] valueContents = (value).getBytes(StandardCharsets.UTF_8); SchemaAndValue sv = jsonConverter.toConnectData(topic, valueContents); @@ -257,10 +256,9 @@ public void testSchematizationStringField() throws JsonProcessingException { @Test public void testColumnNameFormatting() throws JsonProcessingException { - RecordService service = new RecordService(); + RecordService service = new RecordService(true, false); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); - service.setEnableSchematization(true); String value = "{\"\\\"NaMe\\\"\":\"sf\",\"AnSwEr\":42}"; byte[] valueContents = (value).getBytes(StandardCharsets.UTF_8); SchemaAndValue sv = jsonConverter.toConnectData(topic, valueContents); From eec908ef2b75374d72665d8abcb5e90f8c0756e6 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 17 Aug 2023 15:55:07 -0700 Subject: [PATCH 02/42] autoformatting --- .../connector/internal/SnowflakeErrors.java | 5 ++- .../internal/SnowflakeSinkService.java | 1 - .../internal/SnowflakeSinkServiceFactory.java | 5 +-- .../internal/SnowflakeSinkServiceV1.java | 6 ++- .../streaming/SnowflakeSinkServiceV2.java | 9 ++++- .../streaming/TopicPartitionChannel.java | 5 ++- .../connector/records/RecordService.java | 11 +++--- .../connector/internal/SinkServiceIT.java | 38 +++++++++++++------ 8 files changed, 53 insertions(+), 27 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index a30cc54ac..ec259dd0c 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -262,7 +262,10 @@ public enum SnowflakeErrors { ERROR_5016( "5016", "Invalid SinkRecord received", - "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion is enabled (see " + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + " for more information."), + "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" + + " is enabled (see " + + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + + " for more information."), ERROR_5017( "5017", "Invalid api call to cached put", "Cached put only support AWS, Azure and GCS."), ERROR_5018("5018", "Failed to execute cached put", "Error in cached put command"), diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java index 4a025a6f4..90c722067 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java @@ -2,7 +2,6 @@ import com.codahale.metrics.MetricRegistry; import com.google.common.annotations.VisibleForTesting; -import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; import java.util.Collection; diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java index fad9bac59..0e35556d0 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java @@ -1,11 +1,9 @@ package com.snowflake.kafka.connector.internal; -import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; - import java.util.HashMap; import java.util.Map; import org.apache.kafka.common.TopicPartition; @@ -25,7 +23,8 @@ public static SnowflakeSinkServiceBuilder builder( SnowflakeConnectionService conn, IngestionMethodConfig ingestionType, Map connectorConfig) { - return new SnowflakeSinkServiceBuilder(conn, ingestionType, connectorConfig == null ? new HashMap<>() : connectorConfig); + return new SnowflakeSinkServiceBuilder( + conn, ingestionType, connectorConfig == null ? new HashMap<>() : connectorConfig); } /** Builder class to create instance of {@link SnowflakeSinkService} */ diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index 9e1130f08..e733af34a 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -94,8 +94,10 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { // Setting the default value in constructor // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) this.behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( - connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( + connectorConfig.getOrDefault( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); } /** diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 142784c4b..798a86304 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -118,10 +118,15 @@ public SnowflakeSinkServiceV2( this.connectorConfig = connectorConfig; - this.enableSchematization = Boolean.parseBoolean(connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + this.enableSchematization = + Boolean.parseBoolean( + connectorConfig.getOrDefault( + SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); this.behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( - connectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + connectorConfig.getOrDefault( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java index ebb939924..3821c7f27 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java @@ -259,7 +259,10 @@ public TopicPartitionChannel( !Strings.isNullOrEmpty(StreamingUtils.getDlqTopicName(this.sfConnectorConfig)); /* Schematization related properties */ - this.enableSchematization = Boolean.parseBoolean(sfConnectorConfig.getOrDefault(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + this.enableSchematization = + Boolean.parseBoolean( + sfConnectorConfig.getOrDefault( + SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); this.enableSchemaEvolution = this.enableSchematization && hasSchemaEvolutionPermission; // Open channel and reset the offset in kafka diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index ee50e8aae..240d8cf55 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -23,7 +23,6 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.internal.KCLogger; import com.snowflake.kafka.connector.internal.SnowflakeErrors; - import java.math.BigDecimal; import java.nio.ByteBuffer; import java.text.SimpleDateFormat; @@ -106,7 +105,11 @@ public RecordService(Map connectorConfig) { connectorConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)); } if (connectorConfig.containsKey(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)) { - this.ingestTombstoneRecords = connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG).equalsIgnoreCase(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + this.ingestTombstoneRecords = + connectorConfig + .get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG) + .equalsIgnoreCase( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); } } @@ -119,9 +122,7 @@ public RecordService(boolean enableSchematization, boolean ingestTombstoneRecord // TESTING ONLY - create empty record service @VisibleForTesting - public RecordService() { - - } + public RecordService() {} public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { metadataConfig = metadataConfigIn; diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index 810c91b96..6c1aa5cbf 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -18,7 +18,6 @@ import java.util.*; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.kafka.common.Configurable; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -53,7 +52,8 @@ public void afterEach() { @Test public void testSinkServiceBuilder() { // default value - SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); assert service.getFileSize() == SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT; assert service.getFlushTime() == SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_DEFAULT; @@ -85,7 +85,10 @@ public void testSinkServiceBuilder() { // connection test assert TestUtils.assertError( - SnowflakeErrors.ERROR_5010, () -> SnowflakeSinkServiceFactory.builder(null, IngestionMethodConfig.SNOWPIPE, null).build()); + SnowflakeErrors.ERROR_5010, + () -> + SnowflakeSinkServiceFactory.builder(null, IngestionMethodConfig.SNOWPIPE, null) + .build()); assert TestUtils.assertError( SnowflakeErrors.ERROR_5010, () -> { @@ -154,7 +157,9 @@ public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() th conn.createStage(stage); Map connectorConfig = new HashMap<>(); - connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) @@ -219,7 +224,9 @@ public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() thr conn.createStage(stage); Map connectorConfig = new HashMap<>(); - connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) @@ -259,12 +266,15 @@ public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() thr } @Test - public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() throws Exception { + public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() + throws Exception { conn.createTable(table); conn.createStage(stage); Map connectorConfig = new HashMap<>(); - connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) @@ -286,9 +296,9 @@ public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConvert TestUtils.assertWithRetry( () -> conn.listStage( - stage, - FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) - .size() + stage, + FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) + .size() == 1, 5, 4); @@ -333,7 +343,9 @@ public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverte conn.createStage(stage); Map connectorConfig = new HashMap<>(); - connectorConfig.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) @@ -939,7 +951,9 @@ public void testSinkServiceNegative() { conn.createTable(table); conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).setRecordNumber(1).build(); + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + .setRecordNumber(1) + .build(); TopicPartition topicPartition = new TopicPartition(topic, partition); service.getOffset(topicPartition); List topicPartitionList = new ArrayList<>(); From 62c1f24855a5f0cde8d9e35d62a9feed12d3db9d Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 17 Aug 2023 17:43:04 -0700 Subject: [PATCH 03/42] works, making it --- .../kafka/connector/ConnectorConfigTest.java | 156 ++++--- .../connector/internal/SinkServiceIT.java | 236 ---------- .../internal/TombstoneRecordIngestionIT.java | 416 ++++++++++++++++++ .../streaming/SnowflakeSinkServiceV2IT.java | 4 + 4 files changed, 514 insertions(+), 298 deletions(-) create mode 100644 src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java index d20ce6b2f..839a88acb 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java @@ -7,6 +7,7 @@ import static com.snowflake.kafka.connector.internal.TestUtils.getConfig; import static org.junit.Assert.assertEquals; +import com.google.common.collect.ImmutableSet; import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; @@ -16,11 +17,50 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.kafka.connect.storage.Converter; +import java.util.Set; + +import org.apache.kafka.connect.storage.StringConverter; import org.junit.Assert; import org.junit.Test; public class ConnectorConfigTest { + public enum CommunityConverterSubset { + STRING_CONVERTER("org.apache.kafka.connect.storage.StringConverter", new org.apache.kafka.connect.storage.StringConverter()), + JSON_CONVERTER("org.apache.kafka.connect.json.JsonConverter", new org.apache.kafka.connect.json.JsonConverter()), + AVRO_CONVERTER("io.confluent.connect.avro.AvroConverter", new io.confluent.connect.avro.AvroConverter()); + + private final String name; + public final Converter converter; + + CommunityConverterSubset(String name, Converter converter) { + this.name = name; + this.converter = converter; + } + public String toString() { + return this.name; + } + }; + + public enum CustomSfConverter { + JSON_CONVERTER("com.snowflake.kafka.connector.records.SnowflakeJsonConverter", new com.snowflake.kafka.connector.records.SnowflakeJsonConverter()), + AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY("com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry", new com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry()), + AVRO_CONVERTER("com.snowflake.kafka.connector.records.SnowflakeAvroConverter", new com.snowflake.kafka.connector.records.SnowflakeAvroConverter()); + + private final String name; + public final Converter converter; + + CustomSfConverter(String name, Converter converter) { + this.name = name; + this.converter = converter; + } + + public String toString() { + return this.name; + } + } + @Test public void testConfig() { Map config = getConfig(); @@ -656,81 +696,73 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - Utils.validateConfig(config); - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.json.JsonConverter"); - Utils.validateConfig(config); - - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "io.confluent.connect.avro.AvroConverter"); - Utils.validateConfig(config); - - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - Utils.validateConfig(config); + Arrays.stream(CommunityConverterSubset.values()).forEach(converter -> { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, + converter.toString()); + Utils.validateConfig(config); + }); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.json.JsonConverter"); - Utils.validateConfig(config); + Arrays.stream(CommunityConverterSubset.values()).forEach(converter -> { + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, + converter.toString()); + Utils.validateConfig(config); + }); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "io.confluent.connect.avro.AvroConverter"); - Utils.validateConfig(config); } @Test public void testInvalidKeyConvertersForStreamingSnowpipe() { - try { - Map config = getConfig(); - config.put( - SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); - config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "com.snowflake.kafka.connector.records.SnowflakeJsonConverter"); + Map config = getConfig(); + config.put( + SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); + config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - Utils.validateConfig(config); - } catch (SnowflakeKafkaConnectorException exception) { - assert exception - .getMessage() - .contains(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD); - } + Arrays.stream(CustomSfConverter.values()).forEach(converter -> { + try { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, + converter.toString()); + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, + "org.apache.kafka.connect.storage.StringConverter"); + + Utils.validateConfig(config); + } catch (SnowflakeKafkaConnectorException exception) { + assert exception + .getMessage() + .contains(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD); + } + }); } @Test public void testInvalidValueConvertersForStreamingSnowpipe() { - try { - Map config = getConfig(); - config.put( - SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); - config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "com.snowflake.kafka.connector.records.SnowflakeJsonConverter"); + Map config = getConfig(); + config.put( + SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); + config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - Utils.validateConfig(config); - } catch (SnowflakeKafkaConnectorException exception) { - assert exception - .getMessage() - .contains(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD); - } + Arrays.stream(CustomSfConverter.values()).forEach(converter -> { + try { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, + "org.apache.kafka.connect.storage.StringConverter"); + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, + converter.toString()); + + Utils.validateConfig(config); + } catch (SnowflakeKafkaConnectorException exception) { + assert exception + .getMessage() + .contains(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD); + } + }); } @Test diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index 6c1aa5cbf..b8e2b0883 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -151,242 +151,6 @@ public void testIngestion() throws Exception { // assert !conn.pipeExist(pipe); } - @Test - public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception { - conn.createTable(table); - conn.createStage(stage); - - Map connectorConfig = new HashMap<>(); - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) - .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) - .build(); - - SnowflakeConverter converter = new SnowflakeJsonConverter(); - SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - - SinkRecord record1 = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); - service.insert(Collections.singletonList(record1)); - TestUtils.assertWithRetry( - () -> - conn.listStage( - stage, - FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) - .size() - == 1, - 5, - 4); - service.callAllGetOffset(); - List files = - conn.listStage( - stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)); - String fileName = files.get(0); - - assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); - assert FileNameUtils.fileNameToPartition(fileName) == partition; - assert FileNameUtils.fileNameToStartOffset(fileName) == offset; - assert FileNameUtils.fileNameToEndOffset(fileName) == offset; - - // wait for ingest - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); - - ResultSet resultSet = TestUtils.showTable(table); - LinkedList contentResult = new LinkedList<>(); - while (resultSet.next()) { - contentResult.add(resultSet.getString("RECORD_CONTENT")); - } - resultSet.close(); - - assert contentResult.size() == 1; - - ObjectNode emptyNode = MAPPER.createObjectNode(); - assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); - - // change cleaner - TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition) == 0, 30, 20); - - assert service.getOffset(new TopicPartition(topic, partition)) == offset + 1; - - service.closeAll(); - } - - @Test - public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception { - conn.createTable(table); - conn.createStage(stage); - - Map connectorConfig = new HashMap<>(); - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); - - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) - .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) - .build(); - - SnowflakeConverter converter = new SnowflakeJsonConverter(); - SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - - SinkRecord record1 = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); - service.insert(Collections.singletonList(record1)); - Assert.assertTrue( - ((SnowflakeSinkServiceV1) service) - .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition))); - TestUtils.assertWithRetry( - () -> - conn.listStage( - stage, - FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) - .size() - == 0, - 5, - 4); - - // wait for ingest - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); - - ResultSet resultSet = TestUtils.showTable(table); - Assert.assertTrue(resultSet.getFetchSize() == 0); - resultSet.close(); - - service.closeAll(); - } - - @Test - public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() - throws Exception { - conn.createTable(table); - conn.createStage(stage); - - Map connectorConfig = new HashMap<>(); - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) - .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) - .build(); - - JsonConverter converter = new JsonConverter(); - HashMap converterConfig = new HashMap(); - converterConfig.put("schemas.enable", "false"); - converter.configure(converterConfig, false); - SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - - SinkRecord record1 = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); - service.insert(Collections.singletonList(record1)); - TestUtils.assertWithRetry( - () -> - conn.listStage( - stage, - FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) - .size() - == 1, - 5, - 4); - service.callAllGetOffset(); - List files = - conn.listStage( - stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)); - String fileName = files.get(0); - - assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); - assert FileNameUtils.fileNameToPartition(fileName) == partition; - assert FileNameUtils.fileNameToStartOffset(fileName) == offset; - assert FileNameUtils.fileNameToEndOffset(fileName) == offset; - - // wait for ingest - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); - - ResultSet resultSet = TestUtils.showTable(table); - LinkedList contentResult = new LinkedList<>(); - while (resultSet.next()) { - contentResult.add(resultSet.getString("RECORD_CONTENT")); - } - resultSet.close(); - - assert contentResult.size() == 1; - - ObjectNode emptyNode = MAPPER.createObjectNode(); - assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); - - // change cleaner - TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition) == 0, 30, 20); - - assert service.getOffset(new TopicPartition(topic, partition)) == offset + 1; - - service.closeAll(); - } - - @Test - public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() - throws Exception { - conn.createTable(table); - conn.createStage(stage); - - Map connectorConfig = new HashMap<>(); - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) - .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) - .build(); - - JsonConverter converter = new JsonConverter(); - HashMap converterConfig = new HashMap(); - converterConfig.put("schemas.enable", "false"); - converter.configure(converterConfig, false); - SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - - SinkRecord record1 = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); - service.insert(Collections.singletonList(record1)); - Assert.assertTrue( - ((SnowflakeSinkServiceV1) service) - .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition))); - TestUtils.assertWithRetry( - () -> - conn.listStage( - stage, - FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition)) - .size() - == 0, - 5, - 4); - - // wait for ingest - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); - - ResultSet resultSet = TestUtils.showTable(table); - Assert.assertTrue(resultSet.getFetchSize() == 0); - resultSet.close(); - - service.closeAll(); - } - @Test public void testNativeJsonInputIngestion() throws Exception { conn.createTable(table); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java new file mode 100644 index 000000000..8ecc8fc9f --- /dev/null +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -0,0 +1,416 @@ +package com.snowflake.kafka.connector.internal; + +import com.snowflake.kafka.connector.ConnectorConfigTest; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; +import com.snowflake.kafka.connector.Utils; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; + +import java.nio.charset.StandardCharsets; +import java.util.*; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.storage.Converter; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TombstoneRecordIngestionIT { + @Parameterized.Parameters(name = "ingestionMethod: {0}, converter: {1}") + public static Collection input() { + return Arrays.asList(new Object[][] { + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, + {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter} + }); + } + + private IngestionMethodConfig ingestionMethod; + private Converter converter; + + public TombstoneRecordIngestionIT(IngestionMethodConfig ingestionMethod, Converter converter) { + this.ingestionMethod = ingestionMethod; + this.converter = converter; + } + + private final int partition = 0; + private final String topic = "test"; + + private SnowflakeConnectionService conn; + private String table; + + // snowpipe + private String stage; + private String pipe; + + @Before + public void setup() { + this.table = TestUtils.randomTableName(); + + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { + this.conn = TestUtils.getConnectionService(); + this.stage = Utils.stageName(TestUtils.TEST_CONNECTOR_NAME, table); + this.pipe = Utils.pipeName(TestUtils.TEST_CONNECTOR_NAME, table, partition); + } else { + this.conn = TestUtils.getConnectionServiceForStreaming(); + } + } + + @After + public void afterEach() { + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { + conn.dropStage(stage); + conn.dropPipe(pipe); + } else { + + } + + TestUtils.dropTable(table); + } + + + @Test + public void testDefaultTombstoneRecordBehavior() throws Exception { + Map connectorConfig = TestUtils.getConfig(); + + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { + conn.createTable(table); + conn.createStage(stage); + } else { + connectorConfig = TestUtils.getConfForStreaming(); + } + + // set default behavior + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) + .setRecordNumber(1) + .addTask(table, new TopicPartition(topic, partition)) + .build(); + + // make tombstone record + SchemaAndValue input = converter.toConnectData(topic, null); + long offset = 0; + SinkRecord record1 = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); + + // test insert + service.insert(Collections.singletonList(record1)); + + // verify inserted + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + TestUtils.assertWithRetry( + () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); + + service.closeAll(); + } + + @Test + public void testIgnoreTombstoneRecordBehavior() throws Exception { + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { + conn.createTable(table); + conn.createStage(stage); + } + + // set default behavior + Map connectorConfig = new HashMap<>(); + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); + + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) + .setRecordNumber(1) + .addTask(table, new TopicPartition(topic, partition)) + .build(); + + // make tombstone record + SchemaAndValue record1Input = converter.toConnectData(topic, null); + long record1Offset = 0; + SinkRecord record1 = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "test", record1Input.schema(), record1Input.value(), record1Offset); + + // make normal record + SchemaAndValue record2Input = converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); + long record2Offset = 1; + SinkRecord record2 = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "test", record2Input.schema(), record2Input.value(), record2Offset); + + // test inserting both records + service.insert(Collections.singletonList(record1)); + service.insert(Collections.singletonList(record2)); + + // verify only normal record was ingested to stage + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + TestUtils.assertWithRetry( + () -> service.getOffset(new TopicPartition(topic, partition)) == record2Offset + 1, 20, 5); + + service.closeAll(); + } + + + + // +// @Test +// public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception { +// conn.createTable(table); +// conn.createStage(stage); +// +// Map connectorConfig = new HashMap<>(); +// connectorConfig.put( +// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, +// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); +// +// SnowflakeSinkService service = +// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) +// .setRecordNumber(1) +// .addTask(table, new TopicPartition(topic, partition0)) +// .build(); +// +// SnowflakeConverter converter = new SnowflakeJsonConverter(); +// SchemaAndValue input = converter.toConnectData(topic, null); +// long offset = 0; +// +// SinkRecord record1 = +// new SinkRecord( +// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); +// service.insert(Collections.singletonList(record1)); +// TestUtils.assertWithRetry( +// () -> +// conn.listStage( +// stage, +// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) +// .size() +// == 1, +// 5, +// 4); +// service.callAllGetOffset(); +// List files = +// conn.listStage( +// stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); +// String fileName = files.get(0); +// +// assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); +// assert FileNameUtils.fileNameToPartition(fileName) == partition0; +// assert FileNameUtils.fileNameToStartOffset(fileName) == offset; +// assert FileNameUtils.fileNameToEndOffset(fileName) == offset; +// +// // wait for ingest +// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); +// +// ResultSet resultSet = TestUtils.showTable(table); +// LinkedList contentResult = new LinkedList<>(); +// while (resultSet.next()) { +// contentResult.add(resultSet.getString("RECORD_CONTENT")); +// } +// resultSet.close(); +// +// assert contentResult.size() == 1; +// +// ObjectNode emptyNode = MAPPER.createObjectNode(); +// assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); +// +// // change cleaner +// TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); +// +// assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; +// +// service.closeAll(); +// } +// +// @Test +// public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception { +// conn.createTable(table); +// conn.createStage(stage); +// +// Map connectorConfig = new HashMap<>(); +// connectorConfig.put( +// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, +// String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); +// +// SnowflakeSinkService service = +// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) +// .setRecordNumber(1) +// .addTask(table, new TopicPartition(topic, partition0)) +// .build(); +// +// SnowflakeConverter converter = new SnowflakeJsonConverter(); +// SchemaAndValue input = converter.toConnectData(topic, null); +// long offset = 0; +// +// SinkRecord record1 = +// new SinkRecord( +// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); +// service.insert(Collections.singletonList(record1)); +// Assert.assertTrue( +// ((SnowflakeSinkServiceV1) service) +// .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); +// TestUtils.assertWithRetry( +// () -> +// conn.listStage( +// stage, +// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) +// .size() +// == 0, +// 5, +// 4); +// +// // wait for ingest +// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); +// +// ResultSet resultSet = TestUtils.showTable(table); +// Assert.assertTrue(resultSet.getFetchSize() == 0); +// resultSet.close(); +// +// service.closeAll(); +// } +// +// @Test +// public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() +// throws Exception { +// conn.createTable(table); +// conn.createStage(stage); +// +// Map connectorConfig = new HashMap<>(); +// connectorConfig.put( +// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, +// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); +// +// SnowflakeSinkService service = +// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) +// .setRecordNumber(1) +// .addTask(table, new TopicPartition(topic, partition0)) +// .build(); +// +// JsonConverter converter = new JsonConverter(); +// HashMap converterConfig = new HashMap(); +// converterConfig.put("schemas.enable", "false"); +// converter.configure(converterConfig, false); +// SchemaAndValue input = converter.toConnectData(topic, null); +// long offset = 0; +// +// SinkRecord record1 = +// new SinkRecord( +// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); +// service.insert(Collections.singletonList(record1)); +// TestUtils.assertWithRetry( +// () -> +// conn.listStage( +// stage, +// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) +// .size() +// == 1, +// 5, +// 4); +// service.callAllGetOffset(); +// List files = +// conn.listStage( +// stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); +// String fileName = files.get(0); +// +// assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); +// assert FileNameUtils.fileNameToPartition(fileName) == partition0; +// assert FileNameUtils.fileNameToStartOffset(fileName) == offset; +// assert FileNameUtils.fileNameToEndOffset(fileName) == offset; +// +// // wait for ingest +// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); +// +// ResultSet resultSet = TestUtils.showTable(table); +// LinkedList contentResult = new LinkedList<>(); +// while (resultSet.next()) { +// contentResult.add(resultSet.getString("RECORD_CONTENT")); +// } +// resultSet.close(); +// +// assert contentResult.size() == 1; +// +// ObjectNode emptyNode = new ObjectMapper().createObjectNode(); +// assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); +// +// // change cleaner +// TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); +// +// assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; +// +// service.closeAll(); +// } +// +// @Test +// public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() +// throws Exception { +// conn.createTable(table); +// conn.createStage(stage); +// +// Map connectorConfig = new HashMap<>(); +// connectorConfig.put( +// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, +// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); +// +// SnowflakeSinkService service = +// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) +// .setRecordNumber(1) +// .addTask(table, new TopicPartition(topic, partition0)) +// .build(); +// +// JsonConverter converter = new JsonConverter(); +// HashMap converterConfig = new HashMap(); +// converterConfig.put("schemas.enable", "false"); +// converter.configure(converterConfig, false); +// SchemaAndValue input = converter.toConnectData(topic, null); +// long offset = 0; +// +// SinkRecord record1 = +// new SinkRecord( +// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); +// service.insert(Collections.singletonList(record1)); +// Assert.assertTrue( +// ((SnowflakeSinkServiceV1) service) +// .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); +// TestUtils.assertWithRetry( +// () -> +// conn.listStage( +// stage, +// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) +// .size() +// == 0, +// 5, +// 4); +// +// // wait for ingest +// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); +// +// ResultSet resultSet = TestUtils.showTable(table); +// Assert.assertTrue(resultSet.getFetchSize() == 0); +// resultSet.close(); +// +// service.closeAll(); +// } + + + int getStageSize(String stage, String table, int partition0) { + return conn.listStage( + stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) + .size(); + } + +} diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 868b587cc..cb561c5a9 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -1361,6 +1361,10 @@ public void testStreamingIngestion_invalid_file_version() throws Exception { public void testTombstoneIngestion() throws Exception { Map config = TestUtils.getConfForStreaming(); SnowflakeSinkConnectorConfig.setDefaultValues(config); + config.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + conn.createTable(table); // opens a channel for partition 0, table and topic From b39a6914efb31c2fca0e5b5baf91731c2c8426eb Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 17 Aug 2023 17:43:08 -0700 Subject: [PATCH 04/42] autoformatting --- .../kafka/connector/ConnectorConfigTest.java | 122 ++-- .../connector/internal/SinkServiceIT.java | 3 - .../internal/TombstoneRecordIngestionIT.java | 570 ++++++++++-------- 3 files changed, 377 insertions(+), 318 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java index 839a88acb..b28ac882f 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java @@ -7,7 +7,6 @@ import static com.snowflake.kafka.connector.internal.TestUtils.getConfig; import static org.junit.Assert.assertEquals; -import com.google.common.collect.ImmutableSet; import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; @@ -18,18 +17,20 @@ import java.util.List; import java.util.Map; import org.apache.kafka.connect.storage.Converter; -import java.util.Set; - -import org.apache.kafka.connect.storage.StringConverter; import org.junit.Assert; import org.junit.Test; public class ConnectorConfigTest { public enum CommunityConverterSubset { - STRING_CONVERTER("org.apache.kafka.connect.storage.StringConverter", new org.apache.kafka.connect.storage.StringConverter()), - JSON_CONVERTER("org.apache.kafka.connect.json.JsonConverter", new org.apache.kafka.connect.json.JsonConverter()), - AVRO_CONVERTER("io.confluent.connect.avro.AvroConverter", new io.confluent.connect.avro.AvroConverter()); + STRING_CONVERTER( + "org.apache.kafka.connect.storage.StringConverter", + new org.apache.kafka.connect.storage.StringConverter()), + JSON_CONVERTER( + "org.apache.kafka.connect.json.JsonConverter", + new org.apache.kafka.connect.json.JsonConverter()), + AVRO_CONVERTER( + "io.confluent.connect.avro.AvroConverter", new io.confluent.connect.avro.AvroConverter()); private final String name; public final Converter converter; @@ -38,15 +39,22 @@ public enum CommunityConverterSubset { this.name = name; this.converter = converter; } + public String toString() { return this.name; } }; public enum CustomSfConverter { - JSON_CONVERTER("com.snowflake.kafka.connector.records.SnowflakeJsonConverter", new com.snowflake.kafka.connector.records.SnowflakeJsonConverter()), - AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY("com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry", new com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry()), - AVRO_CONVERTER("com.snowflake.kafka.connector.records.SnowflakeAvroConverter", new com.snowflake.kafka.connector.records.SnowflakeAvroConverter()); + JSON_CONVERTER( + "com.snowflake.kafka.connector.records.SnowflakeJsonConverter", + new com.snowflake.kafka.connector.records.SnowflakeJsonConverter()), + AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY( + "com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry", + new com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry()), + AVRO_CONVERTER( + "com.snowflake.kafka.connector.records.SnowflakeAvroConverter", + new com.snowflake.kafka.connector.records.SnowflakeAvroConverter()); private final String name; public final Converter converter; @@ -697,20 +705,21 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Arrays.stream(CommunityConverterSubset.values()).forEach(converter -> { - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - converter.toString()); - Utils.validateConfig(config); - }); - - Arrays.stream(CommunityConverterSubset.values()).forEach(converter -> { - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - converter.toString()); - Utils.validateConfig(config); - }); + Arrays.stream(CommunityConverterSubset.values()) + .forEach( + converter -> { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.toString()); + Utils.validateConfig(config); + }); + Arrays.stream(CommunityConverterSubset.values()) + .forEach( + converter -> { + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.toString()); + Utils.validateConfig(config); + }); } @Test @@ -721,22 +730,23 @@ public void testInvalidKeyConvertersForStreamingSnowpipe() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Arrays.stream(CustomSfConverter.values()).forEach(converter -> { - try { - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - converter.toString()); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - - Utils.validateConfig(config); - } catch (SnowflakeKafkaConnectorException exception) { - assert exception - .getMessage() - .contains(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD); - } - }); + Arrays.stream(CustomSfConverter.values()) + .forEach( + converter -> { + try { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.toString()); + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, + "org.apache.kafka.connect.storage.StringConverter"); + + Utils.validateConfig(config); + } catch (SnowflakeKafkaConnectorException exception) { + assert exception + .getMessage() + .contains(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD); + } + }); } @Test @@ -747,22 +757,24 @@ public void testInvalidValueConvertersForStreamingSnowpipe() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Arrays.stream(CustomSfConverter.values()).forEach(converter -> { - try { - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, - "org.apache.kafka.connect.storage.StringConverter"); - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - converter.toString()); - - Utils.validateConfig(config); - } catch (SnowflakeKafkaConnectorException exception) { - assert exception - .getMessage() - .contains(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD); - } - }); + Arrays.stream(CustomSfConverter.values()) + .forEach( + converter -> { + try { + config.put( + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, + "org.apache.kafka.connect.storage.StringConverter"); + config.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, + converter.toString()); + + Utils.validateConfig(config); + } catch (SnowflakeKafkaConnectorException exception) { + assert exception + .getMessage() + .contains(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD); + } + }); } @Test diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index b8e2b0883..16b2043a7 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -14,10 +14,8 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.sql.ResultSet; import java.util.*; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; -import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -26,7 +24,6 @@ import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.After; -import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 8ecc8fc9f..629b6b605 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -4,10 +4,8 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; - import java.nio.charset.StandardCharsets; import java.util.*; - import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -23,19 +21,53 @@ public class TombstoneRecordIngestionIT { @Parameterized.Parameters(name = "ingestionMethod: {0}, converter: {1}") public static Collection input() { - return Arrays.asList(new Object[][] { - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter}, - {IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter} - }); + return Arrays.asList( + new Object[][] { + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + } + }); } private IngestionMethodConfig ingestionMethod; @@ -75,13 +107,12 @@ public void afterEach() { conn.dropStage(stage); conn.dropPipe(pipe); } else { - + } - + TestUtils.dropTable(table); } - @Test public void testDefaultTombstoneRecordBehavior() throws Exception { Map connectorConfig = TestUtils.getConfig(); @@ -146,14 +177,27 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { long record1Offset = 0; SinkRecord record1 = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", record1Input.schema(), record1Input.value(), record1Offset); + topic, + partition, + Schema.STRING_SCHEMA, + "test", + record1Input.schema(), + record1Input.value(), + record1Offset); // make normal record - SchemaAndValue record2Input = converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); + SchemaAndValue record2Input = + converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); long record2Offset = 1; SinkRecord record2 = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", record2Input.schema(), record2Input.value(), record2Offset); + topic, + partition, + Schema.STRING_SCHEMA, + "test", + record2Input.schema(), + record2Input.value(), + record2Offset); // test inserting both records service.insert(Collections.singletonList(record1)); @@ -167,250 +211,256 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { service.closeAll(); } - - // -// @Test -// public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception { -// conn.createTable(table); -// conn.createStage(stage); -// -// Map connectorConfig = new HashMap<>(); -// connectorConfig.put( -// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, -// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); -// -// SnowflakeSinkService service = -// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) -// .setRecordNumber(1) -// .addTask(table, new TopicPartition(topic, partition0)) -// .build(); -// -// SnowflakeConverter converter = new SnowflakeJsonConverter(); -// SchemaAndValue input = converter.toConnectData(topic, null); -// long offset = 0; -// -// SinkRecord record1 = -// new SinkRecord( -// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); -// service.insert(Collections.singletonList(record1)); -// TestUtils.assertWithRetry( -// () -> -// conn.listStage( -// stage, -// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) -// .size() -// == 1, -// 5, -// 4); -// service.callAllGetOffset(); -// List files = -// conn.listStage( -// stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); -// String fileName = files.get(0); -// -// assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); -// assert FileNameUtils.fileNameToPartition(fileName) == partition0; -// assert FileNameUtils.fileNameToStartOffset(fileName) == offset; -// assert FileNameUtils.fileNameToEndOffset(fileName) == offset; -// -// // wait for ingest -// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); -// -// ResultSet resultSet = TestUtils.showTable(table); -// LinkedList contentResult = new LinkedList<>(); -// while (resultSet.next()) { -// contentResult.add(resultSet.getString("RECORD_CONTENT")); -// } -// resultSet.close(); -// -// assert contentResult.size() == 1; -// -// ObjectNode emptyNode = MAPPER.createObjectNode(); -// assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); -// -// // change cleaner -// TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); -// -// assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; -// -// service.closeAll(); -// } -// -// @Test -// public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception { -// conn.createTable(table); -// conn.createStage(stage); -// -// Map connectorConfig = new HashMap<>(); -// connectorConfig.put( -// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, -// String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); -// -// SnowflakeSinkService service = -// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) -// .setRecordNumber(1) -// .addTask(table, new TopicPartition(topic, partition0)) -// .build(); -// -// SnowflakeConverter converter = new SnowflakeJsonConverter(); -// SchemaAndValue input = converter.toConnectData(topic, null); -// long offset = 0; -// -// SinkRecord record1 = -// new SinkRecord( -// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); -// service.insert(Collections.singletonList(record1)); -// Assert.assertTrue( -// ((SnowflakeSinkServiceV1) service) -// .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); -// TestUtils.assertWithRetry( -// () -> -// conn.listStage( -// stage, -// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) -// .size() -// == 0, -// 5, -// 4); -// -// // wait for ingest -// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); -// -// ResultSet resultSet = TestUtils.showTable(table); -// Assert.assertTrue(resultSet.getFetchSize() == 0); -// resultSet.close(); -// -// service.closeAll(); -// } -// -// @Test -// public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() -// throws Exception { -// conn.createTable(table); -// conn.createStage(stage); -// -// Map connectorConfig = new HashMap<>(); -// connectorConfig.put( -// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, -// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); -// -// SnowflakeSinkService service = -// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) -// .setRecordNumber(1) -// .addTask(table, new TopicPartition(topic, partition0)) -// .build(); -// -// JsonConverter converter = new JsonConverter(); -// HashMap converterConfig = new HashMap(); -// converterConfig.put("schemas.enable", "false"); -// converter.configure(converterConfig, false); -// SchemaAndValue input = converter.toConnectData(topic, null); -// long offset = 0; -// -// SinkRecord record1 = -// new SinkRecord( -// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); -// service.insert(Collections.singletonList(record1)); -// TestUtils.assertWithRetry( -// () -> -// conn.listStage( -// stage, -// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) -// .size() -// == 1, -// 5, -// 4); -// service.callAllGetOffset(); -// List files = -// conn.listStage( -// stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); -// String fileName = files.get(0); -// -// assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); -// assert FileNameUtils.fileNameToPartition(fileName) == partition0; -// assert FileNameUtils.fileNameToStartOffset(fileName) == offset; -// assert FileNameUtils.fileNameToEndOffset(fileName) == offset; -// -// // wait for ingest -// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); -// -// ResultSet resultSet = TestUtils.showTable(table); -// LinkedList contentResult = new LinkedList<>(); -// while (resultSet.next()) { -// contentResult.add(resultSet.getString("RECORD_CONTENT")); -// } -// resultSet.close(); -// -// assert contentResult.size() == 1; -// -// ObjectNode emptyNode = new ObjectMapper().createObjectNode(); -// assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); -// -// // change cleaner -// TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); -// -// assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; -// -// service.closeAll(); -// } -// -// @Test -// public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() -// throws Exception { -// conn.createTable(table); -// conn.createStage(stage); -// -// Map connectorConfig = new HashMap<>(); -// connectorConfig.put( -// SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, -// SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); -// -// SnowflakeSinkService service = -// SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) -// .setRecordNumber(1) -// .addTask(table, new TopicPartition(topic, partition0)) -// .build(); -// -// JsonConverter converter = new JsonConverter(); -// HashMap converterConfig = new HashMap(); -// converterConfig.put("schemas.enable", "false"); -// converter.configure(converterConfig, false); -// SchemaAndValue input = converter.toConnectData(topic, null); -// long offset = 0; -// -// SinkRecord record1 = -// new SinkRecord( -// topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); -// service.insert(Collections.singletonList(record1)); -// Assert.assertTrue( -// ((SnowflakeSinkServiceV1) service) -// .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); -// TestUtils.assertWithRetry( -// () -> -// conn.listStage( -// stage, -// FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) -// .size() -// == 0, -// 5, -// 4); -// -// // wait for ingest -// TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); -// -// ResultSet resultSet = TestUtils.showTable(table); -// Assert.assertTrue(resultSet.getFetchSize() == 0); -// resultSet.close(); -// -// service.closeAll(); -// } - + // @Test + // public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception + // { + // conn.createTable(table); + // conn.createStage(stage); + // + // Map connectorConfig = new HashMap<>(); + // connectorConfig.put( + // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + // + // SnowflakeSinkService service = + // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, + // connectorConfig) + // .setRecordNumber(1) + // .addTask(table, new TopicPartition(topic, partition0)) + // .build(); + // + // SnowflakeConverter converter = new SnowflakeJsonConverter(); + // SchemaAndValue input = converter.toConnectData(topic, null); + // long offset = 0; + // + // SinkRecord record1 = + // new SinkRecord( + // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), + // offset); + // service.insert(Collections.singletonList(record1)); + // TestUtils.assertWithRetry( + // () -> + // conn.listStage( + // stage, + // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) + // .size() + // == 1, + // 5, + // 4); + // service.callAllGetOffset(); + // List files = + // conn.listStage( + // stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); + // String fileName = files.get(0); + // + // assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); + // assert FileNameUtils.fileNameToPartition(fileName) == partition0; + // assert FileNameUtils.fileNameToStartOffset(fileName) == offset; + // assert FileNameUtils.fileNameToEndOffset(fileName) == offset; + // + // // wait for ingest + // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + // + // ResultSet resultSet = TestUtils.showTable(table); + // LinkedList contentResult = new LinkedList<>(); + // while (resultSet.next()) { + // contentResult.add(resultSet.getString("RECORD_CONTENT")); + // } + // resultSet.close(); + // + // assert contentResult.size() == 1; + // + // ObjectNode emptyNode = MAPPER.createObjectNode(); + // assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); + // + // // change cleaner + // TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); + // + // assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; + // + // service.closeAll(); + // } + // + // @Test + // public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception + // { + // conn.createTable(table); + // conn.createStage(stage); + // + // Map connectorConfig = new HashMap<>(); + // connectorConfig.put( + // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + // String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); + // + // SnowflakeSinkService service = + // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, + // connectorConfig) + // .setRecordNumber(1) + // .addTask(table, new TopicPartition(topic, partition0)) + // .build(); + // + // SnowflakeConverter converter = new SnowflakeJsonConverter(); + // SchemaAndValue input = converter.toConnectData(topic, null); + // long offset = 0; + // + // SinkRecord record1 = + // new SinkRecord( + // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), + // offset); + // service.insert(Collections.singletonList(record1)); + // Assert.assertTrue( + // ((SnowflakeSinkServiceV1) service) + // .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); + // TestUtils.assertWithRetry( + // () -> + // conn.listStage( + // stage, + // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) + // .size() + // == 0, + // 5, + // 4); + // + // // wait for ingest + // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); + // + // ResultSet resultSet = TestUtils.showTable(table); + // Assert.assertTrue(resultSet.getFetchSize() == 0); + // resultSet.close(); + // + // service.closeAll(); + // } + // + // @Test + // public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() + // throws Exception { + // conn.createTable(table); + // conn.createStage(stage); + // + // Map connectorConfig = new HashMap<>(); + // connectorConfig.put( + // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + // + // SnowflakeSinkService service = + // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, + // connectorConfig) + // .setRecordNumber(1) + // .addTask(table, new TopicPartition(topic, partition0)) + // .build(); + // + // JsonConverter converter = new JsonConverter(); + // HashMap converterConfig = new HashMap(); + // converterConfig.put("schemas.enable", "false"); + // converter.configure(converterConfig, false); + // SchemaAndValue input = converter.toConnectData(topic, null); + // long offset = 0; + // + // SinkRecord record1 = + // new SinkRecord( + // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), + // offset); + // service.insert(Collections.singletonList(record1)); + // TestUtils.assertWithRetry( + // () -> + // conn.listStage( + // stage, + // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) + // .size() + // == 1, + // 5, + // 4); + // service.callAllGetOffset(); + // List files = + // conn.listStage( + // stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); + // String fileName = files.get(0); + // + // assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); + // assert FileNameUtils.fileNameToPartition(fileName) == partition0; + // assert FileNameUtils.fileNameToStartOffset(fileName) == offset; + // assert FileNameUtils.fileNameToEndOffset(fileName) == offset; + // + // // wait for ingest + // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + // + // ResultSet resultSet = TestUtils.showTable(table); + // LinkedList contentResult = new LinkedList<>(); + // while (resultSet.next()) { + // contentResult.add(resultSet.getString("RECORD_CONTENT")); + // } + // resultSet.close(); + // + // assert contentResult.size() == 1; + // + // ObjectNode emptyNode = new ObjectMapper().createObjectNode(); + // assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); + // + // // change cleaner + // TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); + // + // assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; + // + // service.closeAll(); + // } + // + // @Test + // public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() + // throws Exception { + // conn.createTable(table); + // conn.createStage(stage); + // + // Map connectorConfig = new HashMap<>(); + // connectorConfig.put( + // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + // + // SnowflakeSinkService service = + // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, + // connectorConfig) + // .setRecordNumber(1) + // .addTask(table, new TopicPartition(topic, partition0)) + // .build(); + // + // JsonConverter converter = new JsonConverter(); + // HashMap converterConfig = new HashMap(); + // converterConfig.put("schemas.enable", "false"); + // converter.configure(converterConfig, false); + // SchemaAndValue input = converter.toConnectData(topic, null); + // long offset = 0; + // + // SinkRecord record1 = + // new SinkRecord( + // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), + // offset); + // service.insert(Collections.singletonList(record1)); + // Assert.assertTrue( + // ((SnowflakeSinkServiceV1) service) + // .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); + // TestUtils.assertWithRetry( + // () -> + // conn.listStage( + // stage, + // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) + // .size() + // == 0, + // 5, + // 4); + // + // // wait for ingest + // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); + // + // ResultSet resultSet = TestUtils.showTable(table); + // Assert.assertTrue(resultSet.getFetchSize() == 0); + // resultSet.close(); + // + // service.closeAll(); + // } int getStageSize(String stage, String table, int partition0) { return conn.listStage( stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) .size(); } - } From 290aad2a3422593f25ea640bc546ea90506cb71d Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 17 Aug 2023 18:21:50 -0700 Subject: [PATCH 05/42] passes except for avro --- .../internal/TombstoneRecordIngestionIT.java | 95 +++++++++++-------- 1 file changed, 56 insertions(+), 39 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 629b6b605..d840f2f95 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -3,6 +3,8 @@ import com.snowflake.kafka.connector.ConnectorConfigTest; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; +import com.snowflake.kafka.connector.dlq.InMemoryKafkaRecordErrorReporter; +import com.snowflake.kafka.connector.internal.streaming.InMemorySinkTaskContext; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import java.nio.charset.StandardCharsets; import java.util.*; @@ -24,37 +26,37 @@ public static Collection input() { return Arrays.asList( new Object[][] { { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter +// }, +//// { +//// IngestionMethodConfig.SNOWPIPE, +//// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter +//// }, +//// { +//// IngestionMethodConfig.SNOWPIPE, +//// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter +//// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter +// }, +//// { +//// IngestionMethodConfig.SNOWPIPE, +//// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +//// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +// }, { IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter @@ -62,11 +64,11 @@ public static Collection input() { { IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE_STREAMING, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter } +// { +// IngestionMethodConfig.SNOWPIPE_STREAMING, +// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +// } }); } @@ -99,6 +101,10 @@ public void setup() { } else { this.conn = TestUtils.getConnectionServiceForStreaming(); } + + Map converterConfig = new HashMap<>(); + converterConfig.put("schemas.enable", "false"); + converter.configure(converterConfig, false); } @After @@ -129,10 +135,13 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) + .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) + .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) + .addTask(table, topicPartition) .build(); // make tombstone record @@ -144,6 +153,7 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { // test insert service.insert(Collections.singletonList(record1)); + service.callAllGetOffset(); // verify inserted TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); @@ -155,21 +165,27 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { @Test public void testIgnoreTombstoneRecordBehavior() throws Exception { + Map connectorConfig = TestUtils.getConfig(); + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { conn.createTable(table); conn.createStage(stage); + } else { + connectorConfig = TestUtils.getConfForStreaming(); } // set default behavior - Map connectorConfig = new HashMap<>(); connectorConfig.put( SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, connectorConfig) + SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) .setRecordNumber(1) - .addTask(table, new TopicPartition(topic, partition)) + .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) + .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) + .addTask(table, topicPartition) .build(); // make tombstone record @@ -202,6 +218,7 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { // test inserting both records service.insert(Collections.singletonList(record1)); service.insert(Collections.singletonList(record2)); + service.callAllGetOffset(); // verify only normal record was ingested to stage TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); From 240d968949fd6b69f2a9aa960cdeb4c3c625729a Mon Sep 17 00:00:00 2001 From: revi cheng Date: Fri, 18 Aug 2023 15:23:01 -0700 Subject: [PATCH 06/42] it passes --- .../connector/records/RecordService.java | 2 +- .../internal/TombstoneRecordIngestionIT.java | 129 ++++++++++-------- 2 files changed, 73 insertions(+), 58 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 240d8cf55..5b9c4489c 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -88,7 +88,7 @@ public class RecordService { private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); private Map connectorConfig; private boolean enableSchematization; - private boolean ingestTombstoneRecords; + private boolean ingestTombstoneRecords = true; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index d840f2f95..48bfd510a 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -8,9 +8,15 @@ import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import java.nio.charset.StandardCharsets; import java.util.*; + +import io.confluent.connect.avro.AvroConverter; +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.storage.Converter; import org.junit.After; @@ -21,79 +27,60 @@ @RunWith(Parameterized.class) public class TombstoneRecordIngestionIT { + // sf converters only supported in snowpipe @Parameterized.Parameters(name = "ingestionMethod: {0}, converter: {1}") public static Collection input() { return Arrays.asList( new Object[][] { { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter -// }, -//// { -//// IngestionMethodConfig.SNOWPIPE, -//// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter -//// }, -//// { -//// IngestionMethodConfig.SNOWPIPE, -//// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter -//// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter -// }, -//// { -//// IngestionMethodConfig.SNOWPIPE, -//// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter -//// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter -// }, { - IngestionMethodConfig.SNOWPIPE_STREAMING, + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter }, { IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter } -// { -// IngestionMethodConfig.SNOWPIPE_STREAMING, -// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter -// } }); } - private IngestionMethodConfig ingestionMethod; - private Converter converter; + private final IngestionMethodConfig ingestionMethod; + private final Converter converter; + private final boolean isAvroConverter; public TombstoneRecordIngestionIT(IngestionMethodConfig ingestionMethod, Converter converter) { this.ingestionMethod = ingestionMethod; - this.converter = converter; - } - - private final int partition = 0; - private final String topic = "test"; - - private SnowflakeConnectionService conn; - private String table; - - // snowpipe - private String stage; - private String pipe; - - @Before - public void setup() { + this.isAvroConverter = converter.toString().toLowerCase().contains("avro"); this.table = TestUtils.randomTableName(); + // setup connection if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { this.conn = TestUtils.getConnectionService(); this.stage = Utils.stageName(TestUtils.TEST_CONNECTOR_NAME, table); @@ -102,18 +89,34 @@ public void setup() { this.conn = TestUtils.getConnectionServiceForStreaming(); } + // setup converter Map converterConfig = new HashMap<>(); + if (this.isAvroConverter) { + SchemaRegistryClient schemaRegistry = new MockSchemaRegistryClient(); + this.converter = new AvroConverter(schemaRegistry); + converterConfig.put("schema.registry.url", "http://fake-url"); + } else { + this.converter = converter; + } converterConfig.put("schemas.enable", "false"); - converter.configure(converterConfig, false); + this.converter.configure(converterConfig, false); } + private final int partition = 0; + private final String topic = "test"; + + private SnowflakeConnectionService conn; + private String table; + + // snowpipe + private String stage; + private String pipe; + @After public void afterEach() { if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { conn.dropStage(stage); conn.dropPipe(pipe); - } else { - } TestUtils.dropTable(table); @@ -202,8 +205,20 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { record1Offset); // make normal record + byte[] normalRecordData = "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8); + if (isAvroConverter) { + SchemaBuilder schemaBuilder = + SchemaBuilder.struct() + .field("int16", Schema.INT16_SCHEMA); + + Struct original = + new Struct(schemaBuilder.build()) + .put("int16", (short) 12); + + normalRecordData = converter.fromConnectData(topic, original.schema(), original); + } SchemaAndValue record2Input = - converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); + converter.toConnectData(topic, normalRecordData); long record2Offset = 1; SinkRecord record2 = new SinkRecord( @@ -220,7 +235,7 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { service.insert(Collections.singletonList(record2)); service.callAllGetOffset(); - // verify only normal record was ingested to stage + // verify only normal record was ingested to table TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); TestUtils.assertWithRetry( () -> service.getOffset(new TopicPartition(topic, partition)) == record2Offset + 1, 20, 5); From a2c53e61a2b49a34e8f629d58b451a92dd4ca789 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Fri, 18 Aug 2023 16:09:22 -0700 Subject: [PATCH 07/42] trying ut --- sf.log.1 | 79748 ++++++++++++++++ .../SnowflakeSinkConnectorConfig.java | 2 +- .../kafka/connector/SnowflakeSinkTask.java | 12 - .../internal/SnowflakeSinkServiceV1.java | 7 +- .../streaming/SnowflakeSinkServiceV2.java | 12 +- .../connector/records/RecordContentTest.java | 8 +- 6 files changed, 79764 insertions(+), 25 deletions(-) create mode 100644 sf.log.1 diff --git a/sf.log.1 b/sf.log.1 new file mode 100644 index 000000000..311889f12 --- /dev/null +++ b/sf.log.1 @@ -0,0 +1,79748 @@ +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc8597a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ef7c69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cef81ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d304308 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@724cd85d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48fafe49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5808cc24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@572fec2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@433558a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ca2fe3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d57063 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab3aa8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a3f13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48326a4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@218d5d42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@569d5c0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d236d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa651b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@140a6048 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28cb13cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45204bf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28e6a12e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fa99c5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f0cbf0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583408d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45463d28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ae2296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c203daf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64aacc7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10f102ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c28a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a8ca0a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f77696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@308b55e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56746279 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@298259be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad2c9de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509caf31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258e78f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e7a8a57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32ca2c7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71ae830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@518551d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b675187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78ad2f40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2826130d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d6cc1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@721c448e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63dab2a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bfc7baf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eafced6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e07c1b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38fe3d79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c718ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@407ebb7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c4b0c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39184e51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@210100f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1411cac1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4186021e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a17a6b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1338378b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@965a0ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7882d52f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fa786e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d7172f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9b7dd61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67725ad5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24211015 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e140db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@350ac933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40db89b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c568803 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b9eaca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719e43f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b935348 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ddd6a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68198ba2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e9a282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2161acc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca45edf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28449072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4554da4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@697f49a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4783a0fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b486f7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28ad6ad5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b8c5461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@615287be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69693767 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad1035d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a7753a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63d6387 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26cfae28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a69a47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce0314c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ac420c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac916ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8ddf43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aafd0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@445573d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7699fb4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@524a4518 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d898db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7c8087 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54864de4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5464b0e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12bce4fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4d3696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d42b176 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd0b5cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793e2bbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb54b88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63949289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ae32df9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@628890c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43407e1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@487ac20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7778d5ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51501d52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11a011c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28ff497d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@191b2c7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48f01887 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@594f0665 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b290890 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b91857a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36d88af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22b0da89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619e2bc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2663d8db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49241448 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d64fb77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7db4350f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45a98f62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cc655cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66b3bc67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af3e1f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3328ab50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ddec8f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@567509ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@732c86ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@528c1f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eea18f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@202c6ad0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a08bc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7444f388 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ae77057 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39deff31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a93bd65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5918d70c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18bc71b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d532e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@413ac5cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e05da57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@254f01d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a726da4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4df1ffbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dd64337 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@604aa4ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@291cfa0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd2ec8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6412c53c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@514a40a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f483c29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638235f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11cb279f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12072a46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cf0e55b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eaf54b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76106b23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dee3abd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@167fd666 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b13afb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278d1afd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b9ae763 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63bedc90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388354fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7716fbe1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bebbb9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e24a82e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c222fb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cab1626 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a6a696f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3114c1d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c4627e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38580c61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41802b14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568d5891 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b7944c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@233a60cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a22734 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd7a201 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e999249 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed5b756 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cbcbc3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e19d803 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c100c53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f6c6180 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16987f16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a17406 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a7f9c15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64386bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@537639aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0a6686 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b3d866 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@125fe195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7086311d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d4557dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b23defb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769dd743 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37d20e54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108e4ccc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15df8c20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d606617 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a677941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@486db8ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@213a934c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe419a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e58d06e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b57bd98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e02ed4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61a4d026 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1907a7dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f1a6219 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@468b9832 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e95525 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25f42b83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49928f8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53c2c486 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c9f3a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce56b0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d669e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47573aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5376e441 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@647c4033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1617bc27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3950756c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5de0daeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b9b74bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c60d6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a07fd0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@116b5124 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11d5e921 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51b82b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24a24bb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f8bdc74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a9bc45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4159ad91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7790b9ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4649a316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a1e640e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a76ff9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@240ae318 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fd0b59c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c79a7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b72e5fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1893d9af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3245e374 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@86b8b3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74cae2f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7292036f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd1000a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bb77491 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c99c3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a226d30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d94e32e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d870ff1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49fb5008 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20d9ffea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aeee54c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f338967 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cdbf599 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43127947 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3b20ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d60ecfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee4a7f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e6c212f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59a7484e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53aa4083 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8aa9e0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4700b36f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248666f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252994b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79af34e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b767b74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cd0734d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4535ee61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45bde23e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d189c26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627dc4d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@851681b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2257d73d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b2796fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27f09416 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24622143 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3890f4cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f171d5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c646c8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39393c7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b0910c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11da817b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ef58fc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4edfbe88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce56eae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9f1ff10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1be8fa61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f03c6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f87a7c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4908d85b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ef08e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cd57552 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e1e4c8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415b8f13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c9e7f91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b76f0b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b09944 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a8ebdb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45b33b81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49cc88da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab5421e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@253925f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@790a6de6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54ada045 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e8ad7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589200af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@725d3306 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b4b0a80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab9cdd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e62558 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6938fd01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a34f5ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e17303f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d181840 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63e62417 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e223452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9037d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4be19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56c0e408 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79efda26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b15480f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a901702 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ba14afc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c989d44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0cfbe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@176ecf4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68981f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e235edf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@644a3d48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bf9a22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff3255b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b624fdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e778385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e060e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@116f5cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8eafde7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a033cd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e20b20f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2541a5e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a1b716c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@696d1767 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2296608f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a509fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2be10f99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a36e3ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b357c7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3596395b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16a865b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b9ed744 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd88cd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f003216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a50799c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3709a365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68682f67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@766620fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f332b37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@455ba53f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce1b9e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bc70556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d8bf98a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e1d759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74ae6298 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675034ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3545d627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0594c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29b88933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fa93fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54463c92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56e7590f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bfba3f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d29bf6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@586ea6f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ada3d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52af7a98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228d6565 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff92ec6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@569397f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fc15c05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36aeb8d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70673fd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a81321e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd5603f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c7e1100 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@452e66a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@760cbdd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123cffb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eb60ae1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589b2ab9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b6ae64e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ede79d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60618deb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61aebdbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d6a6f51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@303791ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cd54cf6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4802498d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@663cb73b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a09e3d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ccdb098 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d9ccad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45df3e22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606d5a31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11d57fd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@196165e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25f366c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc51c9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1554ed69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@708ec79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f36701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed59d69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd8107f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c734eb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e9ae359 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb60536 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30a1b937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66713f1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f4888a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@131ba9ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e4ed29f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a89195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5abe0c88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a5bbec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca759cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49064df5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63f6c211 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a357f58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c2a2ce2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b8a577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20b13db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7335737d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e050cc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c84836f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46cbdeef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ebdf3d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f8b7e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd46d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18fb4e2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69290d93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1493c0d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a919294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285923df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93df6d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a21a5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65412d88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7ad24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@628a400c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e9e6cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d28b815 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@249fb819 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a516a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ac45571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7274a82c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bf2890b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175eaaf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f391ab2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48dbe4b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@775e9670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15b5836c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@428ee7e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55676d6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56898ca4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f9edbfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a9a901c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34789a3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0edcbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37d6838b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c2ef1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499b3b63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11a8cdef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12e1de8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c7571cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c250502 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb562dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@673ce6ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70d00094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74591699 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24aa294c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2423a2e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5407d251 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68d6fb96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc34f89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24ced27a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eb119e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11e441e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@165ceb51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4453a6e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26b99d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31333b2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4044583a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19798266 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@470275c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f40cc44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@da7cc1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5818157e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c60dfc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@652b10e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56dfe762 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e74bb22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e0fbfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797167bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503c7b00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b7a678e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78067430 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ab88834 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@396f0618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c140ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2d1950 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@549acb09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15045844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e9bae68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c0ec972 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4e9f79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93f268c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac4312d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@504b3e5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab4d679 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6acf98e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513981c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5889ec7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6432ee09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@722c5ada +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f06a296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@787b511d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bcaaabd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b20db94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4168d874 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b99afd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d784b75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2099cb35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27351c2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f40443 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7728d5cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19067878 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@165df33e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea33baa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f143738 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@257b4b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c469888 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b1f86c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@551cd36d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a40714f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42aa3e57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4504814d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76b0b53b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd378ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c18bd80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d7753a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e9d225f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1acf0f07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b25563 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cfd6d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337793c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5122059b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ba5b91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a392641 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25df7178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6490a685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66f41f55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c36e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b96c38e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad3e2f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c1b4c4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18d4fe3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75b3c419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50524402 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@768d1373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49173692 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22efb0a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60df82ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed7034d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e0419a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51b1fa5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d67c834 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bec67be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258bae61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b36d384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@86392be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60dad59b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18da8747 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55bdcd2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415f5340 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779f20fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3361f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c372e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b22a136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35b4e667 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a81df6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74acad2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a299998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48bcdb64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d2e2828 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cfa135a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a89ae04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34c021ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c33012 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47f8893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5351ff96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@662a5a47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27a8b530 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f6b8a46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a6372 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3756f924 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b5cbcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568f5281 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de3ad1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c17cf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d511477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e64a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2619fa59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bb20ca9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e355704 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de090be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54426e69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f2ef70c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b57109a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220b162e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@657d0520 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d730814 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e0304e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24c3d368 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@291a86d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c4c084a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a2e3127 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304ceb99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d1dff6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61bfb0da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae312c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d243225 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fe07a4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73452e0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71b24faa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6beea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7135cd51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bda800a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48f645c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aaaa48f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@552abfce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a8fafb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ad0741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7c5f07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f2845e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@580d819a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60a212d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dcbfda1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4001e6fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7434be37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4703ede +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5684ed04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19fdc72e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e91720 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dea562d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf76e59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b441a5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b145401 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d03870 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d20115 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@382f77db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51729883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4342965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@472d4a5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff7cce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@410c4a52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64563c17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4904fd70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36e2e192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@159f73c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40fc53e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7dc9f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f3253d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@615d00bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@439c1c15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36e4ffb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1edf60dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@178ef365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34512935 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45b1a9d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@200de030 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40f83a8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5dab24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35ea7a6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a290080 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b846108 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b90f1d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d4fc7da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f50ebf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67044130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b760a4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78747ded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40c31e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50bda8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e14941f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65d40d25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dbc27fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66b8654d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c16b1e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ee266b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6531555b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4006c6bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@956bead +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c31c92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb7027c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58720a39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@646d744c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d13c3df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ba192e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@634a666c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c0ab15d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f249a38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68b53fb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58fc4dc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c59fd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@adb2c88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b52f2f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@717a7c57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8c46a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fe082d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337930a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11436a5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f24afb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24b9528e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ebfa2b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54241a88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1726db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50d42eb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@735d8c0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d938b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ff76c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71afd050 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8d37ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20df8e71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe4d991 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c6bf6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57f17228 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b81c2a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7250ba11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17884ee1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bb41e0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e254917 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39d2f2f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c784c6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d74ba6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63dec7ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d35ec43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6380daac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e373701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bc1d86a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@297462a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cc4889d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b2c213b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eebbebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6400988e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6864f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4687b9bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75aae18e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@158ed35d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a8e62e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@213f7a57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33b596f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@604c766e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49fc99fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134bbf54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a626c73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499e8c57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6400cff3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aff6d38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73279de0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ee64be7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cfb7a9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dbc5495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32bb0e5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4efed29c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a6765e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26219503 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c82be6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7450ff50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4e6302 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@546d6845 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dc5cd68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e9c19ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73c7235d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c575220 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fda784f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7197c7aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b80df0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4db05c0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71144df5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e716eea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b4be85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c2bc2c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f035556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab06e62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd6ad27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ef95ba4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5504c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32bfaabe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b39965b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f49d8e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c09f486 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23ac59e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64608883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@757fda2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5671bdc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17d385ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77173344 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c323930 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27f61cc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c830b7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@624080bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f56de7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18d1c35e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cbde3d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a1efba7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73024990 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65dfbf9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26bf6b8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b417f3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a6eeb0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d5808b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f6b7444 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f435c03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22f8f46c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce7f960 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@84a79ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d5c8fa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e247f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b1d8974 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@504314e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9bbb0d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43aca3de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241dc4f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c312093 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38d808ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eccee5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@543cb307 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@316c34ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490af414 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304d5d67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15874e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c1c10e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff52576 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1509ef8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b476ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8a204e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e9a5e13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b599f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@404080d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@585d3918 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@648b9ec6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c69c993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38eb80ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9cffb32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310a2963 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31d4b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671446dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11743766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d5451a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18fd51eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1ace87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d650d45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c37866 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c3155c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f2695b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aecb631 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2741ef42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf47703 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4def5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb6adfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a564b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb14f29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39e4c95c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63e77ae7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f0b1ac6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a56d409 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5674036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0f323f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a85ce72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@265fa5b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@190637ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dbc606f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@196ecff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c7a373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a54e776 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34355d4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4b2556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2468c977 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@694ae5db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770b22bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7568d89e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a98435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2973cf26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7431676 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34dc9228 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f8f30aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5832d882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47dd8554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17401255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc6a77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe45406 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b89f8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73fdfbf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1190b829 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c399a79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cdd06b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40e34cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5426e57c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64fc47d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a02568 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7c2d41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69922477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f562449 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e8cd7ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5571f31b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db8d45e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7392c9cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b7b51b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@714886e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab9b6bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f80d089 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e07c027 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b4d78c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c161d75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@156b8ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255d7094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7586cfc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29a789b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e22cd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef53980 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d4e3b67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7831e541 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9449273 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@efa93e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12731459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@651ff0bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e53161 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12dace1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1801d1c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2817051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630e3759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27e0bf0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31f9f5f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38731863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad554cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8efa674 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76c7acc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d7127a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56247767 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@238da7d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6867a6c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d2d71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a7fd90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d5a6f1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@685eb7fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7173925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b100f2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25dc9bed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38a009cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c0c1eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d62e746 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c201ef4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34f6a0ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b9f006 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ebb6900 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62601d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631000cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1852f56f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d946c0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ffa10d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c0f47ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@237936d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d03dab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32600360 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6404a001 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ea19da6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d4420ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f0d5f65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e48f22d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c061aef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505b0291 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35c88988 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c18cc86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b57888 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4872b585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3987e1ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43017680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289b3750 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7518615b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73174783 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bea8a17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aba1b8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c9eb938 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@357cc277 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa95cfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40555256 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@498b741d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab77821 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a69913e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6044b5c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72fd1e4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53aa1640 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@150c9d6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5023f881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab683a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49ad84a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383ba737 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7713b741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@946a9ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ceb8c8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490c9439 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23cf9be6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ff845f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f07356c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4207d9e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc86ae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40427373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16389242 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8cc9fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34cd1102 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c73503 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35bf113b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3970e550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e9b67d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd8f8ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a30067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48c73186 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b53cbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c56a7cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72982c18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a278cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3becd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637a4294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50b15a0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40452ccd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@558d0d70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64528bb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee9d410 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5723b444 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6af8c55d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a87e486 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6abeea92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20f5e215 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0b197b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290d6fa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34faea61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b9053a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f045569 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db6070e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7c593b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c8160e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a2aaa83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27b954bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b2064f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6f7bfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@537eb042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e8f0a3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3680eee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7177c34c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@572d8180 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1135afa1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2073ecc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f512629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@544b8bc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@683f6e4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5795da2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3113ce9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@390e46e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad00a8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53edf154 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c259aa4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3757ea86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1df932bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21220706 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7182593c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79099e50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db12f1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c1a1798 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fb2e69e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a7c1c26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e911ba1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22fb773c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a15e3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60a25ba3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6880805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f33dd7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1adc73c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68d14c72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e5fa94d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe651dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@365d21d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@540d59b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb87ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ade4e2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d2b316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b2c86fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40013472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48a6ad62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78dced63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@738afd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed3b9e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3363d0e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c4b841a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e4fc848 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d0048f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10497b65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60525622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@704fcfe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f3776f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147eae24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ef09cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6851ba75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca5f15d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d4b6363 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63f4bc74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@613206d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@358f16eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f92b42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55283255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aa760d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7520f24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dab97b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4905bccd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3855dcf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe9cff1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efef394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a79c152 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cdd26b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d2afc9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6491cafe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6066f46f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@553bbe7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9f765b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55339d6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a1ce29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a61485f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e08dae6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@799e6b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc5a0ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b66327d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@331161d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bd5ec9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d31d7fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577ef5fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c7490f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38211ed7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4c8827 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bbd7791 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b6ab30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33d1ca37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce371da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c728e28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aab59bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1796bead +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ec7024 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8e7bddf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78716bcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab0c1ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39bef798 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c82bc35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fde8179 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23bf2229 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7303f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7409797e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f57ebcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a33066 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503ac3a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc78258 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f8e15fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1312306b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15b38622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4635138e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@194bcf41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd8356b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12d322a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c94fa13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be20307 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@614b1a7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a86c2f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3119f9d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac83ecc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23e9571a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fa480c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18d3981 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae84922 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@335084e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68a998f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e552 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18390d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ff9ed58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb3b670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c1ef821 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9eecce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6d4928 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a7b68e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@515f527d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a8c3c04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c7e1e74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d334768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76542a09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d2354db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31f26ab5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76bd63e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@216e0b58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@569c1622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74996986 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a686b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38436415 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e584839 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3132c52d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7c56af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f3cf4f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e964a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24d5c25a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a923141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a193d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aa51379 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4068cf65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bcbe610 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@757ae59c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17260e91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1133f8f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356c67a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17982293 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30edf477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e9746dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68105ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4a64dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@615b712e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65fe6276 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ffb76d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627a9f7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5892501e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e6974a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af09e02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c47734d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@661fc47b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63563024 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dc8a2c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a026b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72a6460a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20239f6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c5e71e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e7ac28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b13008 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dd5239b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9fda28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76c5b5bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e5cea1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aa1d3b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3b0a52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65883867 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3d2adb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38657e9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e6a9a5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a05caf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f5b4c9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c59624a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566bb3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed99a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17c70acb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b714b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b64c6e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@652a4e84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a9481b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@115e3088 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c209fba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b49a8d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@653e7144 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@504c71b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de1554e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ba8552e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0abf4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e1591eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7969f642 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4051dd64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7899eaef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae1b6de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c0c9826 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6278ca3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55019a0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c19134 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ac4fce2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e49febe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7741d081 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3947b76c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14c96f4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7514df63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c28cc98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@850f6a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57bc7bd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@480598a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e06098e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@416c232e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75baea69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7370a15d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d448cc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d91bb11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fdaf568 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79d6d787 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505d46d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b436fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289dec37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67b87c4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d69e580 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c3b3b07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b8092e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ffeddc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da97059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289f9fe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c71792 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4ede6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48534244 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c67f479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43c30be0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b76104a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19e52a02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a02d17b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@614541d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1acae9f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a4f7593 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eae0a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d6a7a64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228f42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51109138 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e892729 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a45f325 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f093a04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2971ebb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e1631db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bef29e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4da0f184 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b2aa7f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24fdcba7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff6ece +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c999344 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7232a800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38d8e512 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45ac65d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@452fdd9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@223a1f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5db58eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b11ba74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40696388 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32dd02bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@436cfa6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d82a6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea67a7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b457f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@402f768d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3450afae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@392846c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f65adf6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44452789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d9cd70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ebc8b5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce49d63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31708019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a79690a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6075fd69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18026b81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4740d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cb75d4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23667b01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b3428f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f65d689 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69ee892b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79405964 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bd5dbf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a158d94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@472fe02c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c5f1ad4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@593014b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3f5b2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1729e061 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e4c1ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a28f29f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9b0efb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769d781 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7277ad17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f0605cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ed3801 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71713542 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e21316e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31067b03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@607a6012 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1859927a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc46a5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d18c2c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ed15d87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a4fd665 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4655d3e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d303716 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39997a52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d517ffa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f3272b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49572404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bee8c96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc69abe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41f02852 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3177436c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e32f3e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772895fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37da81f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c3dd841 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f9c19d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66cd099d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57be998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5756d6a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3800f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42377903 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@185a6ff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@795869bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@756b89ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6d0963 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4783e2d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dac7e0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5423aefb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3061930d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@682c4a9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ee2db6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ff301e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33604ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9c4c6ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e40c9bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@291b15a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab97519 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35e73144 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d620b76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c25bf6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d651a4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@110465b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c90e55b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d2feb61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68ad25e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab6c49f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@613758af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43e1d30b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e234f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e1c820c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75b54540 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3881a976 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b175739 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2151b4b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e916a9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4beeb627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a919efc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c97507 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43f91cfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d18c528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be8c209 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1674e33e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1de103e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26ebb4f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b3ec2ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a62bde7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f89575a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54be3274 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2968b9e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6caa4ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@639c00af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4614a33a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a5f9585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79639399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7af97d62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19170c74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb2516e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a28ec5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8b8610 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ade5db9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67ba42ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a1d383c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4de19469 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a2bf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3000aedd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53bfbc39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123c38fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ad9ca6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736eb4e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1e90c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e168070 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f471b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b4c3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0a0465 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31050474 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16dbda5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e591e20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@358b02a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3486c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64f5a82a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@625b128a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fa89271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a20fd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4edf43ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74aad7ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fdaf812 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2811e4f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce59ab1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed23b76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56204ed6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71b13eb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b1122c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1798e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a952d0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42299810 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc0d20b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5942a334 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a2c1054 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@444bf819 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49162a11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5017b04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f056540 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3495cb89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c2b39a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42fcb272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4657f008 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c84a2c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49772689 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ebf72f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17111898 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e7a6a50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e495a78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf4ab8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14bea7b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29ab779f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de56248 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e30eb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3201a04a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40e42e1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1645f36c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d18f181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62a41998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3517f28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab632bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20385893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6212edd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a327680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619a92a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@384a2c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@760d8b21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff27516 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a48e1d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c7ccef1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6321adc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af485c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@674fa93d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dbee863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bcce5f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3528f38f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f34b256 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45a3ba23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292c2410 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bea1e31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c6b118 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9ec7ba1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a8c228 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab857ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e127184 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c93694 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43f715a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bfa9fe4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d9594e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b29ac97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@88d478 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f127e32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@233d47b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15461a53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ab2797 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fc4303e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160dc48e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bcc09e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a1dee1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@479a5c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f4bc9e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aa656a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc7cbc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4829af51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29cc7b6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7107038d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127d6f90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@172b09a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5df60f20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a47478 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ddc9893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2275b2c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1726a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78279c8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71b68b18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e5ac45c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@88ed718 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c09f162 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9a25db0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21f200fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fba25cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3b4db9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@476fd453 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@762bc9a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78310ee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee1d2dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47154c1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fdbeb87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f6d96ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6feac8a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f20cf8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32c558f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ee68f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78783f93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a7f69a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e5bedd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@407eb9d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@794392e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6860964a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@372ccecd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2291189 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67184fa5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b2647ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aadbac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10425d40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35024d8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c2354f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3befac24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c55e44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0c3aeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@81e5131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed6429a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d72127a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22d6a05d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583beaef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2485b1e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a996c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eea17e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7901e648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70024239 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b89527 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc39d2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d0607b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff78faa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e773ede +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@193aeeda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24bc098b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77e55869 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30230985 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e216d65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a15d37b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67eee240 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f29b019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a96b6dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8c24fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee03f8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c4bf930 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33649833 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c80f175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4323a53f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20425612 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3770e729 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c80b3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12d4fd19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2094db97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@106d3903 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59bbdf0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cb2dd15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a7cfb77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1320621a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e48cdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e44b123 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63ee58df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73ced60a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43fc2ed8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@87a522d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1943cb5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c3820f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e905748 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6421918d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63bfb777 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3edd6c11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb5695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8e02b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74141491 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@217c43e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@795bd6a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635a90f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@373eb095 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@741eb6e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d78000 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64bd4f81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2645e19b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27edc23a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48559147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@461a1e8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f5a76ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ef8129 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d4e71a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@742c438d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68dd86c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12c1a43f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57e86d4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ce7c11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45a1394a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ade9d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4829b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c54fa9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30f3f734 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6fa6c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1165011f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3e91bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@655b9598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6616647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f35e26d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b83746 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@387685bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a89fab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@259e35f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65dc340b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698b0669 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18af9934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c4a5625 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48b25fcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793ba0fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d39f0d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35c762c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ca2d8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d526a38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3fba32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb244ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@471c9ee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29713f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b4dfea0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2605481a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d019b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2560132b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@314c51d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d059a72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a91590 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7575ade5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2664f80b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@372330bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3ea65e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77fc3cd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d9268d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58514621 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2871723 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@148be95e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bd9580c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@796b7318 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ef5d19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8e58d23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@691dbd0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b4cf083 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72727b66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565a7c9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@522b66bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4f5c95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f87b171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1abc8093 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132b33c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17846ec5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35aa597d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b6cff43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19378d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3555cc65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52531a03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee961ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6426e33c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f8b2d39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35c2a3c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@115a95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53fd76a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75397b0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5545428e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f8a6f41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@677a0a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5925c4e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@268bbc71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af72f86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6b6b8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a9ee644 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118da752 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21521fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44640d3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59d57f31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@259ee46e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eaf0bd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a46f2c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18183447 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a770ebe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf04d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61fd2070 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@381e8a04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aee6147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74cc5122 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3a116b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3038ea42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e5415b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f365404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33abafc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44b6390d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67fa4759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ac1b9d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@247acae1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f1e132 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6404b4da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@234cae85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@519355cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f35d6be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5611e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8a8b62f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@435ea7d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627f192b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cd8eda3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@518c39d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67da3457 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df6cfc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2698902e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16ea27d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2024d1d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be7b2ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@91c565c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e8f0c25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48a3e66e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e054d56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7540567c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@439882d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14df9c5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20840c6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57804db4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5009f3bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38dcb888 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383b2481 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0323df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720881ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fcd6fde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e33498a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ada8648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ceb6f05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489c52d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c373d59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b10051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25f5ca45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f555eb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aad67fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50644972 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4816d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ecd2396 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9e2926 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d43c87e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a6e1d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4160a900 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff96f13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5caa56aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d639c87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@790f1a82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d4a6347 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@335b6d42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12fd580a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13279bdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9982a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c0c129 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d85f693 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d3718d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73ee4fd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60340105 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@224752fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b411022 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ad0001d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a2c1a6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22e47f72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5157ab0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b7d89d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71583195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c6df23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca515fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5134aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb562cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dcc875e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f26dee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b54ee1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb756ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4288c6e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@481f4e47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24577d44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1564fdf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11dfae55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2300b75f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c224326 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9ba23d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62da08bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c6e1f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad3a13b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@398a6c16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b27047a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@789d8db9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fb031e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65b006d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5607cacc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e375f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c523335 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29b10acd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4188e49d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189d45a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450b0614 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64cd9b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e72712 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@102596ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d83095c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e46bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e9a2ebb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62347144 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77685db7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b4c649 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fd4df12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eab21fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9b487a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dbb5fce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74be3724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630d5487 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42f41804 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45367abb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b4722e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4914bfd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bc2bacd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d57eeac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e91d22a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5297180c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c3dc441 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e7d65e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ecde683 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54f0de2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566d173d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33da0a51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62f21b57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30a1b889 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e4b955f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da09e45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f7a1805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22229e93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@709106e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19aafc04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565f6504 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@603faf5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18e12b96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5665be52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b3f96df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e7b39d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774b994c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c272ed9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ba9d8ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5089e95f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b238bfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec723a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b6cfc2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14303f0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73ab1221 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@406f5d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6867f4e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d3a76a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ab493f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65b0a670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3227b6ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c864164 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1068017a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@708bb621 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f94e603 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51b705f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@104ad69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c9329d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74ef66b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d1e5575 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3955dc5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cbd9f3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368eabf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c32a25a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b12048a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb628b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fcf146b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26965dba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4b2bb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67927907 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405955d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5436bbee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40342c45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@349a2f05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e671b81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fef3e68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed867d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bdf850d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d6236e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b584f5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6437126 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21936b0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d8aacce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54912c9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4946bd6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73377456 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b634a22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16cb7a0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a9a205b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22c77c9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@732400ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ef20fc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d178b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c6bf739 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1915de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61bbcfba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f2117e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c30f70a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b80b0b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14797c62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7380a752 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5823968f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76888034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294d78eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f01f0a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b529226 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29e93858 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b2197ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b75d6a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3439a04f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54727568 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@338904af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@265c3c85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9edbd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18802c77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6357ffb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24285f4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d411800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14bb7e4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48cd9937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef96546 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfde981 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64fb8393 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40dbaa1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@256d2412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@611870df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59479074 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44516a3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8508b10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aec8c4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f4d6d54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c06ed10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42469639 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66c574e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a03c9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a7531d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf481fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@715bc0bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f5edd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62f734b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a14aa7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55e5c3ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e07f66d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@758e2d84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a163618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5115c8f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4030e822 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44d61fad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f6b57d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c64e97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67f0c355 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13af9371 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16ffdb42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45f8dfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65541c49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a0f3f89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6242d041 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a0a8198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f7960c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cf9ffd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@584df2d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1605ae03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e8347 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@610ac388 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56666e10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da8a769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b2a7520 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c8be34e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49f80e21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c36b98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290891f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6a9609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ea0263 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290f2c2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e587c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab19b8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49cb71a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@197542f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71461d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37edadfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2539bcf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e82844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@578375f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d5ba8b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65ed73f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3210cd02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ef00a4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ebe7b4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2042e784 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@140e5653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a528cf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37ebc6d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@226968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66781bf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18b31ee6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7187b650 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62fde821 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@447cc01c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44b8b25b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@841b9aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f0f463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27aeb8e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2605ec25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@230c8da6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@215d2648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73e6eb32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a8b45af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d2d96e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@750a886e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31f15286 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1177df2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@403ce95f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61d450ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a8a4b10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752b51e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52fa280f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11eb50be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19b14b58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ba8dfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35836173 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ea6b9dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5ec6da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f695bdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214f2bb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aec061d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719f758c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346d6fc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca7901d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6671bf7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc8852d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3384ec00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6373b515 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8924c6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@205b51bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ab4d5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a0e9dc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50e549d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a4934c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44667543 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a3830a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6965d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63659e0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14de2348 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f589c67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64541386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0e54c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eaff682 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@257fe094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61f30f9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3802bf7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9284518 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53897fe9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aff15f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72bb685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c6033fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@548e7baf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60cec1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc9edc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@117bc27f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368f90fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5782d60b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@443686d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b91cc15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4153b5eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a01144 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70d027b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75a4bcdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2904b369 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292ebd3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5832e20e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8a0cfb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b3ba31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58a820c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17458b4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19162367 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed6046 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19dade8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22fe210a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@adb2329 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@615f8d64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3e0974 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9b9bc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0a2d39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3323a131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d086b88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c6ce83a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33f0776b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d74720d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@338e1e00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18080d73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f37a36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1629813f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5514e3e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@daf1fa0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3ee6e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6444b868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16bd7cc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631a23af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ada888b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b1ce240 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@163ae0a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248fae93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6fe459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566d03b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1713d63b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eb1197c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd971df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5597bfa1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cbc1926 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a008134 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af75c22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5275966a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e3cc02b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@702be5f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@703db192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e05bf39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b00b4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299b2248 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b7b7c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e82a10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118499fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61fd60e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68fd92e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e138d43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bedb804 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d03cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@573cbb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27a837c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@416386a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28096631 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@333c4683 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@554c6113 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8c7f3dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b9c341a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a457fb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c37c516 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd98656 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31ef49d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20463885 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12eb46ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@657799d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e1de57f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f0e6229 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779a8f10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efa3931 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f8d078c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23de2f7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9fd61a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fd35489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78e58d98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a12e825 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54c3d47b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3151b576 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@650c6789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5862f8cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12489fb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8e628c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b878fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675349b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31b0dfc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c77417a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e393af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a005c23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4df9d26e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b53263a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b6b014 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@782f0565 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b46d2ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79a65908 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca57038 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c2150e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f287e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a1a5f48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d59c283 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b3911bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13bda87d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc5555d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@686bd37c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bfc9b3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35fc2c7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1432e821 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22f429e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d7e73f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b521fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a45a90b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@597d70c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27e271e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57281e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2052f839 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@129b7791 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2642416c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310cb2ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a80784 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fc6cf3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bbbbe8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ff6ea83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a45cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10e2c9e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368b6a9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb13c3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6538c79e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6393bf02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e21d650 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29566b42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@af15dc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74648832 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74f743b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e613a1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e7e6d47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e0b4da2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@702f3358 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17794e80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a701d7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ef237cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@674e7cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a1e339e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@308f5a55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f3c47df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b834135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10ca1b18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5addb6a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f2009d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b96dbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ebb2ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41344256 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26255405 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b8c01eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70598edb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@139c5474 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f95bf7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24ad663e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183e0b26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6970c3a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bacefbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14fffe63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c70a0e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f21905 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f9fd5da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44782aac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d76fd6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ecff1f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fafd2f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a07a8e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60109562 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2191b8e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3038e647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672f3025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f01227 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f0ae408 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58773570 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46478843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4525db86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d774d8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4a33e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26b5adbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4da7ab04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4abb8403 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@476343aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cb0c6ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c3fbbb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a32bbad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7711c352 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d5efeba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c7d08b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ad2b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3adf6b32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70bef9fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64927fed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ecf9f22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f08fb32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8fe8a3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f64f1da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4de0579b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9e47e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79aa298c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2198ed42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8b2a65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241bf9de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd54298 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a415cbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d2e9b92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26804ce0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@670b2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60231b2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49f8feb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d3b4c21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@553d34f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d3fb71c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@233cd167 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1451cb65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f7df94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a00ac1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dd24b5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2742a567 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31dfc564 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d1379c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66f6356e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@542dcd4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b16c64e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b3b7f67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb16c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@364ca76a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc5335b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b85a703 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c52406c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f76c37b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9b7e25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b10632a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45cf0b04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c5b75dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a175e25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5256777e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ffcd11b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@86ea4a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a825fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49245a16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40593e20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3defe90b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54819cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9e98d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@363f0938 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74da1945 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c662733 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67f55a5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294de955 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@df9dd08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@103b3c3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dd183ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b10ce42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c251b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11516472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@404e4b57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d31efc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17344228 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ddcd648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f7a3057 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d679785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5877ad03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33f24ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d643872 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6e57e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6914ecdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4831a3f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34eaa524 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ac5052a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f97ab9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@411227bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20b0693e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42bed457 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ddd520b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d22a3c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e78c8c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a50a735 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d7af97e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@650e63a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d088100 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@122efc9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68e556a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e603b76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67f9b53b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d6477b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40440996 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f802163 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15fa0bc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a76b9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@454fde05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7a74f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e9bfc7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@207e9ba9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b7ad655 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@330fdc08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5296a612 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1cc3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b647acd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c60035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@610790c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698f00cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@155ad21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a4c296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a789d73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be0618b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5770f773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d12f79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68eca1d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1078797a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40e3df69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f069aeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e970d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51896a59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fcfff96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3425fd46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d034cdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67422a86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17815ab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd54acc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b578a2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d379bd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fe3e2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26db97d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@718a77d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5042d36e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21bca822 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ba3c6c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793526b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2a435c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b130123 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ec30506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e938cf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce02da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b71199 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46fad3bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347c1ba0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47571727 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f34ed8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fc3aae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bb611a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2427cbd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c470255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ec10fb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65910ce5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f726a8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b839a9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5db9aac8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@302273be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce85135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b78674e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b076103 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289b4c5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3664c181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6efe9214 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e41f9b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ccab79b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cb2134a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c648fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@771bb1e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e14a6f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2290cc3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3605c10e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c3be28d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37b3ccf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13661705 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf3b9d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@481f1daf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e29419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@429f691f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54345593 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64f6d8df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1141da04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@282f9ea2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ec1def9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca3d5de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132dafb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4723e9fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17c5c1f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55afbf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf4a4e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23c0d667 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab37998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e4ac68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1669ea1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bf6559 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3241a50f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2bd8c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cbe839 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e96c014 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71407d14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ac4e3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29fffa33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51910978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23cb6683 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31840973 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@677c5312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eff4c4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4678ca6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57269c43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5247a176 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@161e1a5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b59949 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7592db58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab1988d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c1bad6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46de1c74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25f547ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3396c62f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12c765e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@244ce677 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b091f35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e2b03c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eaa3ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4456926e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27c5dd66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5106c770 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3235999 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73167e71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a31a398 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1debc6e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41dbb5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44b8a858 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33e2ee34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@707dba2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19532827 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1828a93d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9f7742 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ae1e3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21bc083e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c803c75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b6261c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aecc76e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b50dff5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ce3d7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8a3a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72d1550e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f5393d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7851608f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57242f89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f256946 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cab9257 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f8dc255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6923ee4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc8d571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d41dbb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c2f2522 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a08df53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71c0fa3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220f1bc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77dc2c96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb28779 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dd5913b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475e585a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f074bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e86b8b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f6bb811 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a42af2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c54371 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631d2ffa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21f4a8b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e705877 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cfc4e03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4694da99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42210b9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e0e5048 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@702c91dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108ccd52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5abb710b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769bf604 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622d2b7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143d9c30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@150f9495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8d7be6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f0f1f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f543724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2616a967 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7113a408 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41dc849d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fdf7695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c91af98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ce6fe6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7593e10e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ac5af7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1070dc35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33623082 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59132c71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2bbe46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c2c5842 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e688f65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ed85e7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2946146e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4746a74c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d3f1b66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@138830a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18440227 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a4aaaa4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f7b78a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a9e8c96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4273fcd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd361b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1989b8da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@177d7d41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a35d64a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c8db015 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78968236 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd7f10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@601e7e43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d2674d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d19c673 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3213f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118cab44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23565e37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f3e70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b3e0d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f1c5335 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eaa0f07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f021c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc6a58b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@614ed6dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f323fc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3022abdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6991aaae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e7e717f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20ecbd6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5359ee1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61958033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c6df0f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29af1ff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c45eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9dd285 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e746989 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ff59f0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de643aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eccb769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7399ad18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40275198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd3a125 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34df3049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a90baa7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c181fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2249cc48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f59fd30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@609327fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca18471 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70663021 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50f6f360 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29a38257 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39b1191e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7817c131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f9d50a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a99c83e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b515d9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d891988 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28e45ac9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd8bfd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d74965f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48249120 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@421ca8ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b33b148 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@328c001f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ecd2aff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b12a89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f5e6d06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d001c0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aad4b87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab8d539 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6652b935 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4f09b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b1e8824 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47f39363 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@341b11b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577ad442 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@186a2921 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43d9c9c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fa39edf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efa87ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67909072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a0af2ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@596cdc55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23d1ef3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40a1e832 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d83f511 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b20080 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e368fd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21190a5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43505810 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11b4e6fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ba6bb08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22040bd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63555759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5560eeff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b6071a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29b8bd9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@609ec86f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@325bd595 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a1e4f0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a6459f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606d2694 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c76a071 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6478e0c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20b6cb6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29e6006e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10aa480b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33f7fc01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@250c9c8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c701d31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e58e0a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72431c9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@336a06b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f4f0c79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@362b80f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c66fd67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f5a2fdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4881c08b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2247d54f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49a199a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@276b738b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61287a75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd936b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f0e41d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fb3fa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66220e9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52fad84b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d950e05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@708e3233 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62eac6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2921461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4732c710 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ac6a6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3592fb01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27b1d0ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ea996f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53307acf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d8c95b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5846400e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a458b88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a73600d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dd6f609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77fa938d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b55f31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10f73002 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3abbfaab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19fe708a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12b9f853 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be7bd26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eec6653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34fc2724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69da2a88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f52b418 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5231995f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a1413a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f17b52c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@451f24a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bbab63f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@169f8ce3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1423f5c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a5e9db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6660a9be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf2d905 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f4f8ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1822fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63d6561f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79f68191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59652dc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18abd3ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499c0849 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42be2e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7450d48a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a24556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b55dd0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b21071 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32347869 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@129c138c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27943951 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f5944a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66135ded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@146122d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12b9cfbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49354e65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35358e9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285f4d4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c7b3d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17628d4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e3a986d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c5d99d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15cdc087 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16c6379f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce52cda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cc6af8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f48168 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@202934c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45cf5fe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cbc2a04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe0153a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46868b04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7909c446 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14083d69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3beeb317 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4942d1c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ede175f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b688e98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65d0f6e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ac3df6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34295387 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5a17d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@395fd5e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d435d0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6308904f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68e7cf4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f50a72c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@287c0cc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fd17b60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@591a5070 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73d9e5b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b8bbd47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f85cb2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3417857d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@85f5262 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e58a7ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f95c5ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77218b0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f0bc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed1912f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f3a9bae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43806ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79048caa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bfe224c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b22576 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347b40b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69c7fb41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43814877 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22794ce4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce9b693 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2241bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a9707f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5f0882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a836c54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a4e519 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9641ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69ce4772 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@480e67d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d97cfe6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4594ce7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e7af468 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6571208b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a2b3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65fefaba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c2e9bd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@379baf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c3735e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@457a6836 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d92c313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c4c1ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d510708 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bfce14c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70bd277b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37b14b28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127d7391 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed70613 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c8764d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a1d8f97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@130588be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6514f353 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71fa8ba2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779dd33d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e2e508 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a2e092c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f2c3450 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b11eb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@692477c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@133974a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d794c2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638395b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2953343f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71179180 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66188b94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1949acf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290b8128 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64dd8e18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46bbed3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1241f725 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67362684 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f13d934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76580be7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b3d32d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d693b33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c429fd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a73b28f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@785a8495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c9e980e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54fa2249 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337a1593 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fa85145 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@670ee456 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e37a94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1563dd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c2976c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c951547 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355bf19b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d3d26b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12db2b67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76003b56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cbddb24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e60252 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cad81da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b80d992 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4996513d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed52ecf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e56d39e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a7dd8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c3677f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb781fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@279561b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a039b6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7978f3cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a33256d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a77557 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60195e1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ec12f48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4a71b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@707aa883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a39e927 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dd53647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@138c01bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36979dfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23722ada +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b147fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c33756 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eb12c5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c27fdab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a4c57d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb02398 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b942197 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28ad45eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fc5f2ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6244b42d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1826b96b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a663599 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a56ecf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df4d242 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50b4d176 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b15fc04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36a9febb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d145cee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5d1f5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fbc25e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76994e82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39258f93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4af1f82b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@240c17f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18ddb0a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d11ed3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48647c13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c131130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793ec08f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc49aaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@102bf520 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a5fc84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b907e27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54de8c18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b13f156 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30796f6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38904e02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@188eab44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c937042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@676a6eb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5db0dc02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3009887b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371fff58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33410fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58bf8c56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24898d6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca4439c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb41d16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b4de58a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347c754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4db1dc03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2282240e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2667f461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b27fa9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58b57c3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b248171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7279b488 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@562e5353 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2589993f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ae9abf6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550b9fd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719bfaee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1832c583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f236b68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@365fae7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183d6ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16adae8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3271ef1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79968b60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a2e2977 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@710c0f46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cb81774 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1385808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a26acc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc1eaf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@674e434f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2475afb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b7a1f15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f08dec4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a49b051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1348a4b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f1937a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6146965e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36da691f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e12e59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@442f0b7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118e81c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74f9907e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d03eb5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21cb7769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ae6a18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6c7893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e37255b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bd75cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337421ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c436ff3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b199add +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5b958 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160c937b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21874abe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ff321 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f69a5cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0c7c10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c50fb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f4a1ab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d6f875e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@633d8c1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d6b9f6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@621bb3d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ef9db8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3dacae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d7cfa8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22464fd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dfb147f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c0283b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d0b624 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56f859f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13e5e7bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd2e962 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@787019aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cd3f096 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50373aac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c025031 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c7bb52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7763bc1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e680a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70054222 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49a17a5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2348bc2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7daeaa7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27e0e3bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@655136ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e47fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bebc098 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b362741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47be207b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@366e96e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63d46fbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@794250a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f491a02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d520cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dbb2345 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2617f543 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21096289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca0e6d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28954b03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31b10ebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b463f2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60eb9cb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62241ef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5930f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310cebdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dbec77b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fafcbea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5de39472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b67fe26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7019752e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4158105c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@594c01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347b04e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f70b746 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eab8c04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44d8e094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2868b0f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22a8f835 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b19bc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736e190a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eec90e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be0ba5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f31131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d6ee8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e472550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b428c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1feea135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cff8eca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b5553a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3190e364 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1d0b6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc4571d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4acdcd96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cf09c68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f058ec7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66505485 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@425594a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3abe535a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@592e817c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dbe02a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a04107 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bccfcaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c4abce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ba214b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2adbcabe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@560e5ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45071137 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e6bba97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a911bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e6c9b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31fa37e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70531e88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@224ea4b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3861f620 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@469ba601 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e544704 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79e82565 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e0f0c8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55bf00e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59df0134 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6124c467 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5779dfb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a6c6dbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1b0566 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c5bf59e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54625f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cdea978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f6dafb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c594ec6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68849103 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f87002 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@731a4fa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65a30079 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29638947 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd98409 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a9fb18f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3742d06d +17-08-2023 17:39:01 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2de4e73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25806bed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1224b51b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33c4de8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4319b357 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e077da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52db8a10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477e3e4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49f3aefc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f4b4ebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214eca4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17f4146c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4324cc20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@366f5c4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61a4fc66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56c4d34a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3364507e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f25ff6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39247b25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490aa1c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@587cef37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f28df4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1215329c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e0dc5f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31378656 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db620a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160c6651 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ce2b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc616a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6769c21a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12a21e48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@153361fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e0b0ef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@775b7e3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75896ec2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bbbd507 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d411838 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c72b29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f06b0ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab053a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fc50211 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b3e8a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@deab920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450fee51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@524db96b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a73a571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42883ca6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78cf3fc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47f64753 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40293af1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43c7435e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@427745d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@374e4e1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3904b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@479d7d9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fbcb13c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac70e46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cad8123 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20f04488 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a9f34fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38e8a0d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58ac9a77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5a51ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79619ab2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f24b6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383106a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10ccf56c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@658d49cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7332c729 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4386f3c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3496094d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bdaed33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eb47eee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad8ae2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e800658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@425fc5ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f618065 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a7f870 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e3b86fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42f64d46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a79fd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eac9de9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c02d490 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cee322b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a5554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc3d347 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@514f2efb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c0d3193 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f8e19b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab2acee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72a269b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77264fdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@382a830d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77308d3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16bebda8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a36033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ffda3a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd7b039 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe707f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@678d48c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@353f0f3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@441827dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5357401e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e91d1f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@131a53c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b9b0ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f8745c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc93dbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8cefc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35ebbecc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70874cef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1645b385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fbe945d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77cb54b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de18e9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@603db38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6df4abd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b75c4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71939f99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69d14088 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b38408 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1cc171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@361ef2dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55282197 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68866cfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@552c2fd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1465bfbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d5103f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bef6d47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f89bb73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a86968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e54df6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e7110b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c816a2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6977bffd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31808236 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a2e034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108536e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4459fb5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@717d40d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aead88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3825995d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33ad161e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284da360 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cfb890c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48801cbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1346fe57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565509fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566659d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ecbc244 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a5b338c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44bfc7d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20f3b657 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606a89ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e113d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f5d7d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4098fb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30185b29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@432c63ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54706f05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c4c3e02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e7fffb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eecd283 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19b3841c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64f20506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75e1fc5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d3150ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58a7ac75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c894852 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56705291 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@113e4fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b234af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29f3bb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56af05ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48852f86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de76d75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@465e9bd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68724ce4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@96ab773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f60ffee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d523d88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@473d43aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb0b511 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13a7a606 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fb57787 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123843c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cbe74a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228774a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@361a263c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@666c0534 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19887e36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@331bf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67fff60b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61bc0a65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b1fa1aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@236be99d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3c1c55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad2c675 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c063274 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dd1b68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b586192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7795f5e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a4decf6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5223a68c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a6f19ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@120ddfe1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47f2bb0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@82a62be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a3a9051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4634b220 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14cc6b07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f28c4b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@217e7d1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b86ea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355ba1a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cdffeed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e9bf2e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd744e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769ae18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4402108e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75bfce31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@201a7c39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a491f4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e9be4eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355e7b4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d62269 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64c7f680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51afa4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43d87da0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346adad8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f425923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d581e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ded4170 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f33e00f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22c5218b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f10a3fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8a18fa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4852252b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43908cd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1945e021 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d9df06e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44bff58b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30e1fdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@357cb373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47902b32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c388de6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fbbb57d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f8c78d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@190311aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ece8d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@732ea1cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd81d93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed7db83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@83b08e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9df8e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1264827 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@715d9043 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2171bac8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40ca1cec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@731e03a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6edc4a0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42570ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@526a4339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@516a49c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5deac5f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2790ccb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@276320f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24c461bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f8ab5d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e5e29f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3004cc4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@374a8272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d14cd3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32fa6cc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248ab860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4730f791 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c26a77d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d5b6271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c639a02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7caf6e60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@652b33c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73f58df9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5366ba0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54bd026d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68b4b636 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f1749c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512106af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d33c88c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@207bd61a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c92031d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45df6a5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36dbbdc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@486e7606 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44681efd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5b11b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11859e8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a6014fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7b68bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0f0a76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31c3acba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1baea1b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e6504f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1acdeff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3647266b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@580a9a1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0e774e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b498934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b29e48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68642aa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd94e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@221abd4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52aa3799 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd23e5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72218365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6b0b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b34962 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c19af3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72dd3f72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@424e32a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eaa9bdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f0ba766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4363d5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37de8c0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3da0ad6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f521d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4abb2374 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@94660d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31356751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e619f0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d52b730 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77284ee2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d3db0b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ff6f3d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6745382b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52af257f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5217efeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a77ee18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143d011d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f924a38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52178296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f5a1e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bbc6be8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66523b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8bf1643 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@125f5c07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a94968f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d6849a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d954343 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55c28f92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b7c4b4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@445626d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ceea44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123151b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3e76ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e2b9789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a13d8c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b7b8e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a743b1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28717407 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c507c03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13efb7d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8d8ca79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b2a5629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7445872b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1afabf0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54bf3528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5daa88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75f91cd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a34cfbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@170fbba0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57464a23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ae4ebb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76b1b848 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c89df85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797b8965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@348f14d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f0799f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da0026a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3835901a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74507219 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76bfd32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac51ce7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e64eae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79d3918a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@210971ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1319730d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a038049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fb91942 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35558d53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0c01ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2bc5cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4fb6e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4569972e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4867e515 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d77546b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cae199a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@757721c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24f2d30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@657cf51e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@691242e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a73717 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5efab83e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c2c62fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27523ecf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b51aabd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30e1dea1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fe541f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f7b280 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f45f341 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31ef20de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a025ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1047e659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3038c0eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7899efe2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d83a2d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d9f414 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34905b3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34144b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a4eb32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e50c806 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48af254b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c29431 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@156b1a40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d03f013 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb70585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@981713e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4475d354 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368430a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52fb80d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50f11339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af49500 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2de2abc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23111bea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b21ac6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e36a9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d675973 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7602c77e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@367728 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e7f7207 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1934007a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@796d0d15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a51ba27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbb0620 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7635f87d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6721560b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e6584a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@377bd070 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e5430f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8b23d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@119e8e0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@788f3419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3713485a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71ec2ddf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14670336 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1549c25f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38b51934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a74928 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47591b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff5f052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f192e21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d65c0e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75357aaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@532633b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc2d9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@704d87da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e736b7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ceee30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a1ca44e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46738b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736e30ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9eef98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31f0291 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59eab6f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b033e07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@102af8d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b5a6bce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@670b5522 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25babcc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62913cdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aad2f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67b2c368 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@179e2ff0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c029e0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe6b92f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3e9a38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58b9d3a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3023e473 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a760c99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53b0b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e157cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ce291f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4326f87b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eed2eb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ae5d94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a60120f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e71c618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@495eefda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79ac4998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73d6dcc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36785ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d799a25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2682f49b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63f75c1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c82ff28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ddbee2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cccf123 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45025101 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e329d36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ceaa049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@754031e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5089c0ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ac82ca6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6309255e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@446296d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a70b252 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d618a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bdff417 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b5b39b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74e6eca3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1594c9b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11efe858 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630b9524 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@204bfe12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13a162eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de2e5ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ffedc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d3b30e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2595bb97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb2ef57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@729e9a9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16c61014 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e074858 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4697d498 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405bfcbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66bb1211 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42dad316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a11fcc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b05931 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b25fc0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff3de1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a31ccd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6f7370 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299e394d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388c1955 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cef25ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6afd33f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f13bb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a6fca09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f5a42ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cbb9057 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@272c0cd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3387b5be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e07548b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e7e4a34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79738ba7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77d7eb72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33305a0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fd389b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a181b59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52dcff13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630d2765 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@709482 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae88506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff9cd2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6545f9ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6115b514 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcbcc85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5efae6e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14b85caa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c9f63c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7947a644 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7083a25c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2608edea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e78701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@755662ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@182abf4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@792c7619 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@747e39b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f9ec58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe4cdab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e2e0f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60d4e657 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d325d39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e242a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f5cfad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d4617a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8021ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@282183d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30b1b9d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62bcc104 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dcc03fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@395714ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4151d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25484e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d0b7847 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c78c901 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50287ff2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ae52a81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c0abcda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6600569b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c47246 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a6ff5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc37b3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b9354d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d1349a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c5e114c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f0997b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69146f73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e4d459c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12e5fd68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e62950e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64470846 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60a2c0f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d116f36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67e14b7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bb87afd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4deba005 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b550e51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@768204f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e15f748 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6e8507 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dfa271d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b392be9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab72a71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9bcc9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c25d15c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e41cb52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2df69249 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73575c4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@212adc49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c19865 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48bc3cce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@140bcbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76fff04e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b205c44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2534fa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6728e37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f066337 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26edccef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e5ba9cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c4a9b05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10e95a60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58ab7626 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5db92d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43b78ff0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1957f7bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a4ea8f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c8cee43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@333b202d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1577dbe1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a11d8a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c4cf56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b14a70b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1097db5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6437dd38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@137e1097 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dccd10c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@232f9eed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@792b49ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab214b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719de25c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e9ac274 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ddf902b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7423a8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1953b73d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61a24f01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108dc5d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60d232e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3868d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71ca0b97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f97d30d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606c8582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e25b0ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f928a0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eac2548 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ae8ed8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@399982eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7210ad3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d194051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503066c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@254b38f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52c50838 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e9d1f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@363c95cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a4ca752 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30b2e2eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bff9b1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@493fe740 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6666af02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4164bc9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a4f0dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@101716ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@435aa511 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@273e79a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@261d4783 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e978788 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18f3d966 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd458a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393fee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@322053a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@349e946d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae19b4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13ee19b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23e3d50b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631e10e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3573dc20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56b368bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc5a4ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@784f1052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53f651a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ecd25ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5869101c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58a446c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393e4f19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e2be79e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e52e7b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@243a20e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb7f5f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e31f886 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3797d7a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42dfcc9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1217e105 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6393ce09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a306f44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5637c584 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b05098e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69b7361 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f3866b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74b953f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d1a521 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@98cc4a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252d7687 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e94fd70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2735a72e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4873c6a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a40862 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f021abb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5584caa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3947859 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d59751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a4d8692 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@196f80a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39197567 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30ff120a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36807b91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530f8f7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21cf6a31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69685340 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c01f10f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73dad1aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299eb302 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2471722 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@519d17ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74efcbf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@283560c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12f18ceb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b6acb2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d015dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22e0020e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c68446d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4126cb75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f94878 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@422bd61b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c1ea91f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b14a08a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61848b18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d83023d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@349c0ac3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae2ec2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ffb7025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a0c1b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e12dc9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63dce6c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27e2c8cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dac6b4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e2be51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18cfa8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3019fec4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e311d89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e313628 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@188fcbce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671fe9c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1247dfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76538b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1064ff3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@82d6d37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ac40a65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40a18b36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770b191a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@142c8619 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb131db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5205f100 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de9f193 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d85ac79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2d7bfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67dd144f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@558a044e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@754c9ce2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fd313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b3fc0b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e121de4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f282b48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6585b4f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@104a8af4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ea95824 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@508550d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18ef742b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3650e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189a4b32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62639b33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cef578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f9144b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@251851a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bdcec31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4fb707 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@481e9a9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f6c477b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@136d8bc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@434af126 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f3c6196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a6a695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dea1091 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59bd0bcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4511e1bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42046582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff79bcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ea530ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24f318fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45ba4192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26f0a32d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6197df06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1f270f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3e5bef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6974704e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c4ed89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12c79301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9e5395 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6022b088 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719ba1a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fea7b09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0999a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e3df14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e59e649 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52e7ecb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5198c337 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3defbd6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c156f35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@417ac423 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ec2640 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1628fd37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11c35400 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68fc1c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e62970 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6076ab3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b9e9064 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13e7f529 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134af38c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b08580 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b23082 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d0d820 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31428385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@739c8ca8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be28479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48805afd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7bb6c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a958e08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55fb0c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f66f39f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f1b931e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7928ad3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74bd4e05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c6fb106 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f9f5af6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13e26b4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@db1c254 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c77b565 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79ea02f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@795286b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7af248ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b3f0608 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42eb4c22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f6135db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@473eafa1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54a2ee08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3022e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a591952 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58a8f01f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb997ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d3810b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74e2910a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b55af81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8651b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769744f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@161de1eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50f27d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e77ad71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@120bd299 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b607b8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b961e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bc64da6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@715d15d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc1561e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c14b44a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75467313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cf0dbdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@396407f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10fb427b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5663659c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c9772ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1555e708 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff78743 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127556a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25c04454 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f979a43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c39db6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44417238 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63047c00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4062a7a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b2ee73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d2d92af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d091d76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2883e2d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d2740cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c4e0b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2999477f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ebbfdbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@696def84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51696e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66578b85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2740c4f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21570e56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10471535 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a16bf1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd0bdc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72ec3b52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a2a7687 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@253be4be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b6b6364 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58cef7f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d93c20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b3d3cb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5d1af1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14830b1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c230d91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eb97010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6071a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11b75af6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56c82d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@504b99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e23f766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35686ebe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@547320e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56fdb571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57e4e00f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@210f79d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c36011f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57db67d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27cf01b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d542572 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b6ed3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5fd312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72c2cdc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e611956 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38a8deb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b3c5dfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7479d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29f96ac7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57da3457 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e8048e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7216302b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10dbc3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16a5353c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3e9864 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@96a6e60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47609892 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78534997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46100341 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cd36c8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f34c38d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc9260d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8530fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb59c07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56410b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7169b758 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b97b4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733c9e9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5678d9e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a976707 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c3a3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c38497 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@564d029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@549a20dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37b4b5a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e5a68fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d9c1286 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18854520 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a0ef22c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67fe6619 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a0b5042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2815ba66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f7a2da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@412dd8d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e409f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4275524 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6caed608 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ea600e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e06a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cd08d23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46540e27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56aeecf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37fe5d12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f9f775b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9dc79b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b4d505 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d670bcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79569c94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132d23b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dbfd02b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4853813b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ac1699 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f66a509 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cbb1c46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48b50cb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d48d3e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@674c75af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23510c2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43e14c61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285d503c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb8b995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@723e042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ee83683 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf643d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@656b27c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd2ead7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720197c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c57b281 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70be89a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad04136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64916a3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a30f92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1154572b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9157dc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ba1e6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b6842b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5836bdfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aef0073 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c903035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6544beea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@87036f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a9e4314 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4742cbea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7318e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e7ff7af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76018397 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27cd0b26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c07e067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5db4a8c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27d385dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68882e3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9fef94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39592b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38d30272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118312ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1564bc47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3acbac49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2be9f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13349e6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23828ece +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ccd713 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16dd213 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@369dc393 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dedf70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2388e417 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9a573f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14439464 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6391fcc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d04fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19608881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1108df7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d935e7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1206ffe7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f3f49a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@581b9c0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9c71f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac05826 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc750a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb86296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f7b5cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2feeca7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7af9c579 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527da462 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582e3a3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14abe67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35ac613a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@acd8d59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@239ba1ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77768a7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21858018 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47447634 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ae23ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac873c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c0a9c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d184db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ba85438 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62477afe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10bb49d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f53f046 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16437c00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f0a74bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b57172 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15746042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@185abd90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e995b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a795abd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cb959c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5142c1fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19ce926c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79828f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fa3bef3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6829f7ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5195577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@680cf2d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aea9d05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5b9723 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3226c712 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d3331fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@333f81eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127cea20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e6b12a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32fed6d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d0129bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@859e1d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b156e15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f1965e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b21f56e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb1e896 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c5450d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c610c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c580034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ffeb071 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d2bd95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@95add40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252f35ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f186413 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bf25270 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@641d3893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38417922 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1455aac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d9d0bed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c9756d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2823f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ac80306 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fdfa5d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1446c75a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3325c97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7998afc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b7b6c0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49251a29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e2317e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64384329 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54137091 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3506cc68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ca66277 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa157c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@157923fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@161fd8b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5965a5d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9aaee6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55905a61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d22fa04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@491dcb14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@171abdea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@400eae3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a568ee5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d34bdd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f03bc71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a49bcd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64363cdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c84d7b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e760261 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69aeba6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3caba340 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b8fa16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@387387aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24123700 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45422161 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147b1b95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30919067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e0b42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51137bbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1df865bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2156dccf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@547707bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e73602a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40783bc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57df27a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b798002 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e0ed577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f296f7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1165bd0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@666b46b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e069262 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a20726e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@661111c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5af3bd5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@407ca1e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19961880 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7767b547 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30b9f7a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@156f2bee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9463db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76133483 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@968c10d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6534bb95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aa19653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@157912a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad6c9a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a7e397f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c9b276d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@312a8404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d5dd853 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3603a746 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3965ea4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6679e076 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@555533aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ec787d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f9595d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6646b607 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af20106 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5849e486 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b91c951 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68b4b6aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6244608b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75bec591 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d05136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d9cae60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c1dc760 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f0c113f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d17b009 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6590e065 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47141d70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@683e7944 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58990bb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a684b0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d703eac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@469f981f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@564bcfe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e83de0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72da1029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@483f210d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4003eb08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c00685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52390bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@633f6fef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cbb832b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3f5e17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56435a17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e1b0727 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e597714 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18204b2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54ff39df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@424b2e58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27cf00d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dac563e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1030b195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b926425 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54842c15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b469d1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6f29f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c4d55d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7355b514 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c446016 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b03b886 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b4dfe86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@556e07ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ac59985 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb4c1d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68e9b70d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9d8c89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b022d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7112c5e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54e65514 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@753e62c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f1e5b31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@584484d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e6c343a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38c25178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c85282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d622f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56440a10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ca7d831 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bccd965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73b5836a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d71386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4773255d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad9cb27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a8bbc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@85ec5d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5659504e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@727d2951 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fbd248 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c75cfaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60322b4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b1eb1b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27ab4500 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75f1eeed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@468cb4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a752102 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770bf609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@df7f883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f8778c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cd93d02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@444a352e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76cbeec4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fec1ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@276acddc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@329c6f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b881a03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f0f5b7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e7a6f6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f4d2545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14ce76ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18f13e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cc305a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a826671 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d1e0e92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74775c8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20cda1a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5905ea2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24120502 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6acf8e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27c2b216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5160785a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@656e4d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73475074 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41270e08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13570cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c55e90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d486a8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34667a63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f7cb1cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31635d6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f714dc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38c39d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42cbb125 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69681c2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc263bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78cc97df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6711437a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e31a2c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a1cf07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dcfd283 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70a848d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@334ccbad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9af334 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa275ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7e5019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38baffb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d2a8c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f13aa59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f2df3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5ab7a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2908c668 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b4bf072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d56e12e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70bbe88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68df0387 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce3b68e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b67bad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c13780d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc08933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d9613e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@785860ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f5cbcf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778d2130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a692d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@210d7993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b7ca23f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75e68ac1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2635e131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c3ed585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62ddef67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@232cc17f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a198042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3011fa79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dba8dcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7858943a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e2500e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2158e36c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4865e659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74a0422b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42346644 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2491aeed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4321c308 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3566605d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@548136f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d4e1c48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@303aa96a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36d843b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb3cd3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6612205c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@386b40d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7737f246 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2981ba3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ffcb6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ef3f02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b8250fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4905072b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@422a01cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@780721a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619bb3d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6dc90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12709925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53bb80ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f028465 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5916e749 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb9f4ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@408ba036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5125ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5513b162 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550db933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489a39b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ad60b1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32062d85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7aded7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@133411b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f07624 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a19129a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@649e368 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3543e3ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd8e277 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d47a0d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345c5eb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ea88d1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36cb0fe2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ca48ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20b50967 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aaf8036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1475249e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30dc9bcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@382a5282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e82cb81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36e491b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b8b85a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d6c4e6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@159a5482 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@644de1bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a5ef00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29e5d89f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1ca765 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65d7c324 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c53fd71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b75fb44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@305954c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b4c7f0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4857af61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c6cd2f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@266962ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e55cd33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b8c533f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61cdd1c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cdeb916 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce2c1eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35ce6ee3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a02aa03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@436db25c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@591ace43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f651af5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3018f9a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ec952a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11aeefe9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@262bed9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5109854f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@501b9b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b6df5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a88b187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7367c64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f728102 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6bfb94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b06dd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31d1e459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab019f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f82e069 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29daee9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@171724d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d42d05e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fbee37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b2b6625 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5534c419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e5f3ec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248fed6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@435bd91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7752b502 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@707c341f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bc36bce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2338212 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60abddf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e6c135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@414b59dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff670f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618c7abd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a4f0d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dbad861 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512776fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@537ca7f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52dfef3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@394d4cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@533765a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65420b18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4849b698 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb160a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73435452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62c9be9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35400c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1370d468 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a78e390 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1869cce5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b989d3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e5eddb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1387df8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@455f27fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad11320 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62bac2ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f6c02d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73b0bc01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41cae944 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d15cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a048202 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ca26df7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a3431ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@457ce589 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b167bb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46474f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fc48ed5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db67009 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@357577c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1291765e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2918e0ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27a0ebed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f553bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@204ff15a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ebee2b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ee1970d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d1a51de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2200ba06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b1247b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79a9e220 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5692c86a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e59834 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78436e7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4d3bb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e5c6b8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a57649 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30f6cad4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17c1dfec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a2cc681 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3f0b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4653a55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d42bb8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c66e6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c9d508e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310766da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3991059b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b96a1c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df9991e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774211cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ce6f63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78e85ec9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@496809a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e43a53a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4362e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21677951 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28cb59cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b924399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b46465f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1710997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73062d57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a012625 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53dcf384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6875d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18469756 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b85ba75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f712452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@462e44d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a7e976c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c6ece44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a6b3c2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e57af3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@508c846f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a4a6cd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2963a3f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a3ec7bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e258f6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@375062b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3980bbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fb7076b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dabbb05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@595a19f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1adc3c85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d47ecea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d840dd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34985e4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f5b744e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d49f1f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@603ea923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16a2f156 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24f1ec8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@303ba11b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d9618e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24701c22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e33ff3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33edae9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d566c7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7963faf8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38e800c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@662fd01c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f47529 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ea6faf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b001e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e879c12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2571edd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cd9cadf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6af02bed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58254dad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72901422 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@402da79d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b694336 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59d33bd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b1005cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62c4e7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67bddd7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27707ff8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6528a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68ded0e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6101aea0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45aa82c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@602fb2de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee83fe2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1270fa31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35dfe02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ef915f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21513ee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3465e98e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58a4bcee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ea44cb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26aafd87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d184945 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40a6d1c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ccbc4be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dfc5a65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d17afe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@148fbf91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26d3778a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59832e47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30eca903 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5976ebb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64a4b83b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241553fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e442ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fa14c61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e854ace +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50816843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@320b2af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f724ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6250dfc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@394870bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f7b08f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10849ba4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@744ed88f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ef99323 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a23e8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d4fc1a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f47e020 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d651060 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65cb0317 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46aabcf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@260c12ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@135ff05d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb19a9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3defa6c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb248e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@178392d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3544bff1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ddde36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c416d37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175f7174 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774ef59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c43adef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bfc5385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6744475f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47ea72c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c7830e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fce9028 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a755431 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e1d0f8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b83cb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5facb3dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f38a0c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@94fbdc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c575df6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76df2dfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cbb956c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58ee848f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e20ca89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@211f924a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ff73f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d995fbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5545a604 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@815f324 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225f6125 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@158cbe6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d015f5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a2a0d09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13a50b20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@632aa0ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b6e87b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6effa1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a90b47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1332fac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20f42de8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490e5da0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@249084f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2070a670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ea44fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab2ee3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a2576b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f4314a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7d4629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6133aeab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b2e4ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7753287b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711964b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@da3dac8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad1c20f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42f2fde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea9d739 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4f3fbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@496a51a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34c6d68e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44a6cf30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c0aad5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b3c0e2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9ccdb61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be15a27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e62c18c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4ff10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a3c2b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b7d7418 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@636bb85a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41f5f758 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a1a7087 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a371068 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a0c61c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a81aee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73866578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71308f30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7322372c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14334939 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54de4362 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21620fc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@431bd791 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4def244a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b8cabc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a21854e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@471b66c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7362d719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dbe8028 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b636124 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37d20fca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52eddc36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5ba983 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b01898d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c8a7778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ba1eb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@762bb696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1101043f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e2f90d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec04820 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f4d468 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c4d1815 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e16a25c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42bb76f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61cfad5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15368cc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e69d879 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577c0b84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b45f173 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a342c5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a1e9b71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d098d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a33126 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@723eac90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce574b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d1179d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d9b4d54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a46305b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d41b77a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be88af2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c9e8b36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b1aa16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17db27fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38cb80ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20946bfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a7f94e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6455ba99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@335ea43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@553b9675 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf36f39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b23358 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cf9b8ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f8152b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f5320a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bcf3ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22939b37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37579303 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c499281 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183eb3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@264ad9ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d0e99ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dd3cb14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@409f2c01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@306a005c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69702c16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ac8d147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638e113b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38695f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bfe190c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7720bfdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64346fcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@532a7f9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48872a8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aabaa14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da6c0bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df08618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c520072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b4ef0e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2026661a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@591fe440 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d23e81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59873f1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a828481 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c994475 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@386b4951 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58f4e696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68cd785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@201081cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4253655f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3decfbd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dbc3312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7906cf81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20a63463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ace51f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4656d919 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e37c95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18f754e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10becfd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@605042cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72459d84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50b7a74e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@369132c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@543548e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ac2465f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67220d9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589a1cb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26962d0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3efe673f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4947c383 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b011b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d9bacf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b53e8b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7f6b86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d038a60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@369742cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32050ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b1747e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14d0ba22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@185d035f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21550a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c29d61d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d396810 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d0ec313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65b6326f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cd89fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71c218fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a6f5b8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e1e2de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75f7e583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14933e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24909c1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b8edc58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11ff357 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c719e7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ca19392 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11f0f642 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f8b2a60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f89db93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f14f8f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@594055b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6faeacd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5ab7f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26611daa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd7092e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6705da57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cfb1462 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5259da1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17023706 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e9c90d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a60d766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66755b3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6049faf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a69e427 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26ced3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a2ef083 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@459f3161 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c2ad29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c37d51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d743f22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21dc11d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64324b23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f008b01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@433136ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@222ddef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@743fe5db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f9c30de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d0f197 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51099167 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a93b7a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e776ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@767975ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be650af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c8fc4c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@235087af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e39426e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47030a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15f84fee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32b2df01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664cc10b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3941f596 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e115a6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4affca7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3b0eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c60af22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@184650e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cdeed82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e021811 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e2a94c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d09281b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36606ac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@180c2e88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4af64c0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e9bfcb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1885ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22198868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a77260b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb867c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7490a0bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aa42c8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@790006ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51aea863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@204a6fb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b31ef71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627dafdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b4ee41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35f026c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21ae1d69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ea24dc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f0578b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf789ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a6386f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1613320f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@723730a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35cadd0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8c90a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c24003a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475dc366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241c4c35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61b92125 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4db0c8f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64df64db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22e4e191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7c4ce1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c0a25e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e26ff01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4b653d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@333831f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772eb79a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e198089 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e7b6cfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c3e617d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e2e8907 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce15183 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7938e659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6612558a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6e8efe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab5a5b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74019898 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@85a5a1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fa5bec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17183ef5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ee69e14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1436b77c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a7508d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e333a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a720b13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5ece76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3791dea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4667ac83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c270dcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a74b43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38fab141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43020169 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57df25b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c198158 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1844523 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@525e1533 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c025076 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb11def +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fbfe607 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dea834e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3385e965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@670fc49f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@678099a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e7996ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@722b9481 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b199d67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8376c02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a5b7528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a637e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23af2433 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10cb1b38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c8ce3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36fe30e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6caa04b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2470d0ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee4352 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44504ab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78631941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a374023 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bb27ce4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6df19ed7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d383b39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d6e44e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550350f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34656ba1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c86fe7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671e5577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28550db4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f2437c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cfea532 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19deec37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19e67a15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3caa33f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4504c664 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b5535a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@149d40a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c916256 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f45ae46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d3f5a04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4299262d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@257d6ffa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@768d28d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70f0571e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b53af63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@351f0d4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74225dd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618bce5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@605623f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d6e37d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bc46331 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f80138 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5960ea66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ecce6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d59f80b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25e0c559 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8d2859 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@187116ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a4a2c4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a3fc52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b7ec0df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b28511c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70eb2e62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@762c1d41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b295363 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52d043ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be1efad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d312385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cc2c70b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cdf0423 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b71092 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cfe270a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56be5f01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52c4b7e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62c71ea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f1cb386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bfebd33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c72325a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769a1371 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c001ea7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b92131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f643bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59c3bed8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355a6661 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee17952 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60850f9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24f4582d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@427b4bbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77658148 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f54a1e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3967b31f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d92924 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aed8396 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60ac36c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d78e83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@caf0b34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a48c804 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b53f7a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43a8c1d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@372a943c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b7e7235 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f142e4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab7e409 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@95f0617 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@112822d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f0008f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ac238b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bdce840 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bb26352 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74aa8978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb630ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73b44bb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ba40e65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b65af20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@358ac3f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6432e728 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f8e0aed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d4a893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aa2671a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59dacec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26fc7e75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a00ca09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a3fee4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c808a97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ca5ca4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75568916 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19ee1d86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6142f81e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a06a7ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b598181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2232a741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54c62911 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc0cdb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c121cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bada6ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e468269 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5386655c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f0e87f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a0dc7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13ff40ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73336b4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285953db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@370511af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e14cf83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36958630 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54b0ccb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2737e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@333791d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29092582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8ea94d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e419f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@652b016 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6617b06b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68dda925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415f4e16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@529e942b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@951e2fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cc0f675 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37542605 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11cdc308 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@170dbdf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c75823a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a07fbbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75adbc74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fb57d0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fe14a92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a8e94d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1258d7eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36a195b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@321ec2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df7d09f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c4b745 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1372b3d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fbe7516 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c0e3d56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50fcd70d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56961af8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aecc3f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1725ad7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21575429 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47f0cb21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5f2b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@212b236c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a46dd7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7baeb0d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19cafacd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61028adb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da306ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638533d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dd7a1f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5876b2c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d15f9a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c4042f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770cc939 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a01b58e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18dbb7a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f793de9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e2d623e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@591713b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c34c18c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ab42196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f2d8e6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4682aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c4124f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd725d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c4a9ea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66630c59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32e12cb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54fe7699 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c890d70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6276cb7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59c9ff2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ea41030 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a126019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e6b730c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68cdd360 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36366bd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f1dc7ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a4b3652 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c1bb6a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29767851 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4837ea66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bbdbc94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ae2e3bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ccbe40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@275f0ae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d05b86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cf2a10a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f9346a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ae0d92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355172aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c927ffb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c984067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aec14fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5496ef2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b642fe0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5561d593 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa05808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cac334d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565d0017 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dcb9a02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5076f3ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d1d94b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337e70bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62b4e758 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b8bb0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64e10f06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24da60ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6111cd49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b16aae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dc4d8a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aec06bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5578ef2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fb61b36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14dcc5b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4edff870 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4c6701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13eca815 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61ef0694 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c26ea1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f897fd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d3de33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11b03b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43e571fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67b6d992 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6093a111 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0e4550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58cb38e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54a780e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29413848 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36fadcf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568a1967 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4203f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ee1ef0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e9c9557 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@785beac2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a82ba8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a017c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d8c0446 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a26f8cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d447948 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a1d3d7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@178ea24e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa3c0f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14636cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57564de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a80995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71c8eb90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4234b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b9e57c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e0f0181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582de3bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a5fbf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eecbaa7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4879bbf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f7ea97d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671ca70b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c1e00cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59bd9697 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5382f99c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a43818 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d905289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ac6a82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c262412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb2b976 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9b873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289a2af2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a19d47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb1ef8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3275ca49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b41ab4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52134cc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@212d0484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d16b56a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b4b6a54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33f6649c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a364f42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7283088b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@523ac545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5268991a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a6c904a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@704114dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d5e77c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e310b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf33263 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@646c32d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fd9c723 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45c50ed5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40a05c24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45df247f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@344e1338 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@591000c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b9326b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27d30569 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63401578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0f0fc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@382b58dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eceafc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64e4d4ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2657a6ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44d8ff4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24034627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6680b7bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d7b054c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eda8706 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d6d484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2308be6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc8e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c34cb48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a3058be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cad7658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e425255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5426ac5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c8bebcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4353b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175f7274 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76fa789d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25fc2805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@646329c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8c32be8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5894fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a02d53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1470349c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57dd191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c6b52bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c930d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11082005 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a129aec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5ac1bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d80774 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51000742 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2b2274 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25666b33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64442b4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35af1a91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53f1a6d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d8606c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33dd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d35c4d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132c2c91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1571f598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c05b75b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f75f954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4f188f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@788f38b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2616234e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6669c181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38736a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43d399f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5448de94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c857855 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@460d7b48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@194e6e7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@be0afb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c703941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@718acebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bdd5b40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@154ebaec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48a92335 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff1ebb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e87ba57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7efefab1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcfded0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f8306e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf54245 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d70c18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ef7d2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33dcd957 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252d5203 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28972d72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675e3351 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@329d588c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d6fa996 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c02e7f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ad0859e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2251fb70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2272c5b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b4bb48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27362995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a11cc78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3efaffe2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e87d8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54849ec7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5072fa23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@391e1a20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@389b1480 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4819d7f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7d9f30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@467b5be5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@262478be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20049dcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43e312e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fdbdd0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a3fbfc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e51bbaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fec692 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c78aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@232a2e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70f54d47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ccdf9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64a26e66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16890da7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ccce544 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2919fb77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ebf716c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b4888a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d6c1cf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68fd3025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@458c92f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a6e4f21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c8752d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b511e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@270f18dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f405c5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b653f82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15f14b5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ae770cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37b84747 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b3f71d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a87f34e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b178dcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e606991 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4517a5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@149e418 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7647482f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b568b97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@445a8320 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7dd697 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4637f1c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c2d0a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2991173e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d3cb5f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34fccc5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b94759e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a1db935 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1226209e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@406a90aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78ea35bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f69ff85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c680826 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@181326c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@317da7aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5188f3d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dedf8f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cc9e0aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d5446e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565501e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248d19e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77927169 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d0ebdc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63aaa5c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f24f257 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bd9d7e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23519bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1563e6a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ddd1813 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18beb284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371fc484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d436233 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48abf808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@256c590b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c9e82c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aea379c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d16fd71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af553d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2656af3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a8ebf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73bf70c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43648ef0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a7f569d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3456f76d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce7907a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b139a18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bc8e459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48a69fd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4842a2fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f4632b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0e52b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f04a282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220cbadc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c61efa9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@186cc8b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4226698e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bbe14c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16345af5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@686331d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19785f4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43137ca9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c465cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d83755 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d01a2c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f7bff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51919493 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b4636b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4673c4cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38fdaf7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@651581ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@500f190d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@802fa6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dd13a45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14024279 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6df685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2323a2f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7774b7c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@716322d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22692bae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c63110e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7b5fd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2666c92d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4db9a214 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d631cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc76af0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4df8ffc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d514d6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35f15d5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@538721fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@598be295 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ef6e6ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@215263e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b303ee0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d29f259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64c1bc96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@563eb6a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cb355b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0ac74c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7746fd17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54eedee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f64750 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a0ecae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6022f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78dbb6c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1877b249 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74420a40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2feee1ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a26070a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f6619fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b12165b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13296d14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290d262c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@363b7b01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc00353 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d99232 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75a5fee3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@182b7025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@751c22e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c2d667b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a5d3fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15cc3b52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1535061 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@662ae08e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ee8cdbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8a739f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7141f6f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40de46c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1803ce74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@212c4060 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa4ecfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7e5187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b885232 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28643063 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505fea9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fb27b7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6682a97c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fec6ae7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40486e97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cfda44f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f4737e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6370c3b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@604f2597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11e12a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47cfcfcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2247b989 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3d43cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2394b7e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a9c95cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@508fce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54fad497 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52575a44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c1b8f3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e36b997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cea86aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4076409 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@741569c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f348cdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17538046 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28cb9989 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77cfee71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c16de38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774c204c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33fe8fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b8b228d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4126cada +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57fedcf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfaaef0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b77e48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e300bbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f885fc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51fbe9e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13fc98a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d1f0c9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70840690 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ebe1dc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@970a327 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c6e054f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8e3ed4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32d034d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e35c036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@226deec2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f581cc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9dc34a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cb98c9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f6034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efefae8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ccbfbee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c96519c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6602ecfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f727368 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c25ebe3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52114f4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23051cce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f40c2f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c695a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4da35d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f32baec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ec8695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c31d77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4562f627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f1dd1bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d272ce5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3507b693 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6706bdda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@562718be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52e3469b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad8d17a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8624f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@666e54cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef11104 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32be46a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@283aa19f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce64860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a3917c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a72d161 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2739a5e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29125c26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b478192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f2be42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d8e53b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66c6d743 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214a8118 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3683bdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cd4ab9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cf81d8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@94f114b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b3b656c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@423a97e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e92ee5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fa66d8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f266911 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9b56099 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60bc8ffd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cca546b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7088d002 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@529bc722 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565c79eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583b0647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e05d582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62abbdb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@570d1328 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57043eb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@176d4817 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28940948 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17bfdb21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61b896d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce85708 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143ed394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7db174e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589a23d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6146fd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d4aef0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1234d8f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b39b583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e760bf6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f9491e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bd31d83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d102cd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1337bd38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48770519 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b9b3691 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@771c7a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2901640a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59aee3fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc379e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1206f9f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3085fe8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e1a5a71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e160c0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b7aaf56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2414572b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6811580a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c95773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a750e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0ce7d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3f2b10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40c19f2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5273ef98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b88a87a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e5a77b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62ae8d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f6cb175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7179079a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6309f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a37ffdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509a12f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69e69aa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dde32ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5832fb87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583f4fd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1bda0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9943e2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46351259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6456bcad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce2313a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@442f0fcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a667ac9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0408fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6928d47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3affbb2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23d62e89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78ce81e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a24c30c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d453bb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f75faf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33b89541 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fc81c99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b54105d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be6dfb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c834d14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2310a6c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@789d8c0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255be7d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2336f584 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b639c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11df2881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc0183b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fce5a12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b822c3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f18eee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489b4b6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450896a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f31b9ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3018c21c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@224a0767 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23762818 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@311a2b75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47640768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356741dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27439980 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e25f9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20da5f4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fbc832b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@140880c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c32375 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33913c37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@787f44b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a1edd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7021ab3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aeb418a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4833fe8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5b7d0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12de4b9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@229af5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b9f61a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61be728a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c4f91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec04bb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d37d56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@466b6f64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a880ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f9d223 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@268f2b87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b82707d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5acca08a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d03193a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228fd60e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aa64bba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d17294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@507e67dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c1de47a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34de95d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aa51405 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c6ca03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a9bd575 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aabe9ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@280cc1c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37058017 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2321dbff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76129881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@385e6d7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23c05759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e939efd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca7d93e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27597452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@682957f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14749724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c8d2bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2adfd5d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3864949b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@819f904 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8c49f80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c61ba0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@403d3d54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b2d8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bbf8a74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5fecce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a1b94c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@db8b9eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a036e02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f3fd010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18a7962 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49b5ef59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@807eb13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29991553 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8a286b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da3df64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a4bf0b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b195b27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@617afb5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622150eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@171ae26f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16db53c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20cde24f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@480348b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@84ad240 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44438592 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd3c8cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59c76fe0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7416e4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f761425 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a763ab4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28acf475 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2493ec14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bab139f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@307323c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fb376e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@447bd64d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@575f12b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e9b669a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1afb03c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@287768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fcd1f79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ac703ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e0d413 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b307950 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c419998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e2929df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30273a45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38552fe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7898fc23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f48453e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4634bb80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@578ff6c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@684971f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65a5a875 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aee7fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3f2010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dda3ae9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45f4cd27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34146354 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637cfa66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2d8992 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c53e6a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bca30b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe15c6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cd33d1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff686eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45a3225d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@640cb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ada4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ef0c5d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@329db39c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12bc28f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41e7306d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29d18be9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@768fcc1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@507998e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d1bb51b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58b0e792 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50d73fb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a517c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4936d48b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e53648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f2691bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3efcdacd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4111d315 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ef9ea09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35192700 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d6bb9dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22ad461f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b549e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f685db0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ffdacd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17288486 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eeb3dc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3436073 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@316a97af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@623539b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15470510 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d1faf33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@682d86d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@239ecadc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e5c243a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4a64f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5566ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f489b56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405a1771 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d9abe43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca9f72c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@575059c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47213dc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f71fbc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c927394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@595985bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17d86c6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@678722a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7896f313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6df7d5e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39186730 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67d4f0b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28159937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9e393f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@145f09b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d4e95e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450c1c72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd1822d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61be7264 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6f914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220c8a12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618fb494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7330d793 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@438e8eb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@514d0c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a3efe3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cda3f0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b968766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a7d60d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7efdf20d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4362a8be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ca0ba03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6af11624 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8deced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dfc9c7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a54b64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66073de2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72f94f4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b2f0222 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ff9048 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ac85738 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7273da3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310ee843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589a53fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5063be46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cad3445 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a279f63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70a8165e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3df41783 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@682e0364 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb44585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e1eaceb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cbe8a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b66ee9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17f2181e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6097ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2032ce1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0ce6bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb93dca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@739e9c2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@456ae677 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e38c969 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c2de609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd57055 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@791e7a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@110597cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@129b364a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@520cbb94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c82599 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65ce7ab2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e2c3927 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@194be76d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ed30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@163b0c6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5911fa8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f270e4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284f3ff6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66591eed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304a552b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1738601b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d94d85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68d98dc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ad86ef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eb4ff00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@520d3436 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@761e3c17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ff83382 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46cd8782 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ddaa7e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aa3c2c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4398a34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee2fca6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62fe37c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62fe8cb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aded2f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13600cc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aad39d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c20a32a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2035bf17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3320b1cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c075cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4900b90a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c06a3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72d815a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41855819 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d8171ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ad74bfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78100231 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@133ec8ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15b7f436 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65cfe5e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1db672e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a802f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52144f90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@486e48de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66611a5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e6603db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efeaf48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@146c2af1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a3151ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1427edaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff89489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab61425 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a788cae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619962f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aceee90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60d3c8d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ec02598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e47464 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5943b806 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27287eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ab732e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f0b690b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b4c7764 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff56975 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cd459a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68632806 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@609c9539 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4486f69e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a218ec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf3aa9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fc2c381 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c83f6e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a222836 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c13ec9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d4fa53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668330f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d7bc788 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8f9b88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c2bf068 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23506f9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34bbfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26523ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e2b3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2142b03a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48c14a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f5f17e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f38c6e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bcf7e5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f8266ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5835b09f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41236ab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@518ed150 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4da57ad9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38619aba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62cd21da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45ee7882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@237bb154 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@177b5bcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cb36b0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aaf2a73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5420cf63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1948632f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e24881a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d8bc51f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f82401a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450909de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f572d3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce7acf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6168a3e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca201e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2adaf785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8b762f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f58a506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23894a4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e12f766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e88d885 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@646232c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dd0d724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4430a83f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c440ccc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@945ed89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2277272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43175359 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fe9159b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39e68a4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bcbe6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@791fcd29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8d4a5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b4311d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5efa76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aaf1671 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb46cd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c0a9ec0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368d5e7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3493da43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@295cf2ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c8958a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dea5578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3701d544 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79040251 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c65ac3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@336b775f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a792f83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5336a217 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@263e013 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6adaae8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299c8160 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fdca85e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5222aace +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6db3d8d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2069c384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@add018f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4deeaec0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e6892ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50731344 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3240a1bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ae0919a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70e58dca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@384449a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f3e5254 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c6e68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0b2854 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47986b15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a492d40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58043b4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c49e193 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@588df9cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b5decb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527a74d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ac5272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76588012 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a4ca05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3379dc4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10652795 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220f6d9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ef7f3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b896e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e9a14d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bd3693e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6688fdee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66f04747 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ac611d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6906798f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48c08ecc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cee32ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f00ae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce0c3b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a820b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dbeb9b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b8f37c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8b869 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55e6688f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d3d188c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbe31ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b01b384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc5107b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78298c37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46a6c3b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40bdc57a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e53de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31c22f7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@410989f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb112be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18e60c35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ff49ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3534114a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5277d1b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ad8f86f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38286cb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d0394bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41152afe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb3b933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7850ff0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3692d5d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e384b25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f230bf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1076696b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30ea1afb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c970ea0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b4cec49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28d2e17a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@480f0b59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cf9eef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30bb6f84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f5d81d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f3816a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c25c1e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47095ca9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24b946d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bcb81b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@596351df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5482b24e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a63e9c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3da23135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8c62f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f5dbfaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad56461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3652d597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bd48270 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7926d84f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48615d66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68bfc790 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67770ff5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285c47f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a94b5f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c816c71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2356ccb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@697f5cca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@260b6f04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@519e7d50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44b80084 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57328dd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2abcab78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8ea2782 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c52ba3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f4dc241 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6547bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59eab602 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40f04c66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618ed1f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b7c1ae6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47aaff2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fb18c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12ed40c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@396dd356 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292e00c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@744e202e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c5dad18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b7d38a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c0300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7461c2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@669254b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5418e8ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ee754e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1feb0d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@753d5391 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66f09b60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f1c6f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57fdda23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48c3ba96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a8a0375 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@761aebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e7440b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b365ae0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11dd6d58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc32d8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e8f90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e1624cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@245b35f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26f6f384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5557bac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a88dcfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4701caa0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@567a6b17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e13628f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4280368d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b1db0b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e3a4010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b985c42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@686f23e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e11be50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f75542 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be46912 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a852dbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2965f581 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d0e8aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5964a57b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a8f833d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c40ac10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a939a8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6620bbde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40104a4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19e95e72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fa8d00f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c35666f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e35018f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7cd2f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@340d5ffd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63dc410d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75731d80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e1382a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20bfd829 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@216a311a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c18a7aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7c2202 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@705cc974 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e206d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@464159dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c03f09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@265cbb71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f91fdec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73d2dc7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22bb9d7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@617f2d6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5131c234 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e5fd6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a4bdeeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52c7beb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d38e6bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16ad7ce4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@205545c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dd860bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@271f5e81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa85cda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72a74467 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cd41f14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@162a959 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19346b23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5178901e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da8199a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39945517 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3434c75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16aec463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ba77be5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6000b56b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b97dfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d43372 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@596c1bfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca24655 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ec6aa96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3631579f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f3b094c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21056610 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2ceabc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@261ddbc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@713bc72f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b041afe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6806628 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e8b26a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@612692e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aaeb7d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9a3788 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2833b8ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d578680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3288a6d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c40f178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6901dedf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@282336f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fea9b3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fa6a89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@636aabef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52cfdd85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f78792 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e0e1007 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388de7fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1402369b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71acf89e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fdc4084 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5839c4b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0a97c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bf4ecf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c15d6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304b0153 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3976240d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5509f480 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@798fe179 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11a62682 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f82ef7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34f64bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54dd985 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@257f7d98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da87056 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a81db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ed0f108 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@301dd2f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e893e62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f6c4231 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36dda065 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68895623 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f340f7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1274ad63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49f1e9c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6beb8693 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64ab2b6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53839d2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@157ad175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22120257 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f5fa529 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50fa2113 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1348a300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21fa1b84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ee947f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@429dc7a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@149b3c58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65cc2374 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52360ede +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e3a33b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@338eb9d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@375dfb52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@557f5399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a77650a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf13389 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@642ec82e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4be2aca9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7cdfea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f24c857 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dd91825 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73806ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fef6cc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@690691c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63ed51db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@136b4d83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77068a74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72dbd501 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b313108 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32ff7cb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e905b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d36a558 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ffd7687 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@662ec57c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd3c0cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a7b20a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73aab337 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63bf9d1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22abec09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28962db4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e83009a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@114b0314 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7d6e90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@759f0526 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638e4cdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a94b019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d9a5235 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2021d495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31d88cfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e43e6a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b700a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f1cf178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a561f3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b819c31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ca11de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ad5afce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6050fe5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57078671 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d129ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@229d4505 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d8ed06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e87cfc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62571fe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a2adbc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2898b359 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f19057 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6203189b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e8b13ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7364790c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47232c1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d1e23a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bcccb17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b30717b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752de3d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3962cdf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24eb2db8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf83bf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16d14aed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f3188a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@440864e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@962d814 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15f86df7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ee008a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3346559a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c7b899e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7abade33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e947bbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f7f1373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f385a7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635f5ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8b90b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a8e09e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ce8e52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dff2634 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f530514 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7181a870 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31fd140c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e75a81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@728b988c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eb56c85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5731442b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8dd3f1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@263c8971 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3480877 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e050429 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fc1e538 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df484b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ebc20c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ee5f7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b68b7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@213e51d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7edb875f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44741a78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@855bcde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b369494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797511d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1db6635e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38a09f7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@575d2ece +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6774e76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28757175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1492704e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a87074c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@163ff894 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c2cdff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1255cf09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fa1768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6968b4bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a46215c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d7969a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6468a7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aa73b60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7130a3a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb98853 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@291efc29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af1be51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e286f4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19848dcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a5c566 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7091e8f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dcea3ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4019830a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@297ee09a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d77300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e8c61bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5281a8b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c25f84e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6187c5b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cbbd9b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a01e9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2064612d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f11d295 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de5d2ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e327f7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ddc149d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca8f4c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f0b87e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@322fd2fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f4a50d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@640fe745 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67b87887 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b4ce6b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56fda79f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dd347ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698b62b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@253e586f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ec4c208 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@497e3b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b52764f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c78b46a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1494e76f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c8a10a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4402eb73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d30cc7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e2cf92b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62dfed45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3155cb1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aadaa6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b1edb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e998144 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618bb8b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1602882c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ab22b23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48931690 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74e16918 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a0dc40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da01a84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dd7a836 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3281a13d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@423329f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f5a6f09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3ceacc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@100b3e5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bace10e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4702fd55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f2015f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46902b8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a9816c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a20b690 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a0a3aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72ab22cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58af0c53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6784a5e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5789f749 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c1f3a09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cf26c4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e9b6996 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b0f6b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fa5b1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca5cf28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@937d8f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@833a208 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59c03abb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bfe6224 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c39c5e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@190f1585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4142f1fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e544b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bb1c6e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af3412c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ce89096 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f8b9364 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a48c8a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a0210e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12d64913 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11c5e89e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9427c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ab93a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5795c29f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@692a51eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ce916 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef11cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5789a2bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c1714a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed17465 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28607e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32668dae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66806d1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63774ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@484fa69c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4259c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bcf258d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@729edff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47d0ec7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b000128 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49b1ac88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d54bb35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d2715e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16355c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52246789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ac1e3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7c5970 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@331498b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42fda974 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7796d47b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a4cb420 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d0a7ec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2293d01d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6367fd38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf5a8c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d721e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ef81296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@222abe4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1328b161 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@688b0284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d9c1a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21f6cde8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7703c30e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47ac6048 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe065d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@486e7a4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@219cdbfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59641a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5904420d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6283ef7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25879bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12ab7d82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69c451f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14418384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d00321b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@448e0954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f0e9be4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17cb7422 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd70c2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f2772ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@540d0d16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@182b09d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bbee0a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f984f62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13c1823a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e47412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f93253c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@721d9af3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5216d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1de6ab84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bdad7c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e0c4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78173ef4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@508500f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dacb1f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68350a5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29147931 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ee2622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29b516e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69128668 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63c5a78a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43ba42e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b9466e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f65b997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d2b006 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606e61a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68ecd1fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a577476 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ee1a9ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d91f8ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fcb399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c99354 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f64ece +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a2829f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a48309 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bf9f5da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@581a5c55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f6b8bb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e46ff9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a659cab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70978f46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33e2b4f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ae1070b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb74238 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10160b57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f548b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@342ec83f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52eeea00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72821fb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eae41bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c830366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@918421c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b05fb51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@773ae42c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4316419e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@855f342 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c4e4e6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf5d84b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d9e6efa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a7651e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47565d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4600c297 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28878d13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@207670fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f02cad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd6b5de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bad4622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@484f0c0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51821385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b00242e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a2dfb3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab2b316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bed83fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4166ebf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e73cc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf2134d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c8793be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac898f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c49a75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c3797bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9de3e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622ccfbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e8e6bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61b23574 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52035841 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56563489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@207dedf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5094751e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@181dee1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f0863eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30550397 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@439aac06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3400cf6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8abf53e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10fa0607 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6966282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345fb681 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630ab731 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23004f69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b7abf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f95ec10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69106bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ca1aa0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63098823 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@130c0d3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79a94bb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@612e5137 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec985a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ddf04bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e986c33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d2c233e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f4a49b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5af57c39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17c16aa0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b888eb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26427582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5727c1c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4baf25e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@647de70f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51267067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38e6aa7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ad450e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a8b2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635d6633 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e659d67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@469eab36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13889339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b6c1385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59203b2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a54fc23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d92bb0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46196707 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e38ed85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5340bf85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b220c05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@760e6d39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f83e2b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79e9cc5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1240b546 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a27592 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2704d3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1038505a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c53d9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34fd611 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2556eb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503d5427 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e3aaa73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33814fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab1e2fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@101707e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b554ea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78090afe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d8aa908 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@758a8d5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64bbf51a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d55d89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c1f7cf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ddeea52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4676d3d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38b1e784 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1feef48c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e81159 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65709851 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@524adebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ae81aed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fdfa630 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e8c098a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ab799d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70f2d510 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@628e1318 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca0969a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a6fbf86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cd3a78e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5769df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@464b22c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11f2609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40311e73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355d8e3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9faf33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5248eb07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac2b5bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43b7dc8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@148615fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@103d4b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee3c64e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73cd38b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c0eac0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48f95294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58382c73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@422a877c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12ece313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10836e28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9c193ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f46f4dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a47785c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@613eb912 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64e580a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c2d68f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d796cb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@91bb020 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4133c92a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66fe6b39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3034a79e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64fa8b08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5f6424 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eed77d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b703dc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b62536c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b5c7860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ecaed01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ee472c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d8e9bee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5397947 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc23e04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad3c597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43db6cc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@de6c6b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4099eb13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@446261ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc7f882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672f4ec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6861f044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f5bb193 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3b4d12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@417f4912 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@267b62a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ce98dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@429b1e8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7997f812 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@404b3a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@343cd387 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec9e4c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6858de67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4960f164 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b92b324 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a1c9b56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5499d994 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a3ce9b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14b97242 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ae18da8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@102ea019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da2bad7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32d6fe43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345cb97c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5197f986 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299b9561 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3a42f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d7c8955 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134904b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@660cbb20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18df70ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f06483 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a47a4a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5911d177 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aaa92a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38fc2d03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f79cc3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@83424cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e5c0b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fa87d52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ea14eb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a15e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f32e5b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@330c43e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17fc67c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@401ed5d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18d9ef4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499bacf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dad1063 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@babb21b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c5f97b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48b764de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dcc5ba3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b21196b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be3d59e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1801dda8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67ef8c48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24463b9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@517c45ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47862eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9077c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499f250c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7afd48b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf83ac1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530972fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638e9ad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7326c6ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ec5c47b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c3f7f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310cb9fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d80890 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@642eec6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5debeb7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@97c425b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dd4576c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0a9136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb83f80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@473dded6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58cd8e6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22b43764 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f9d488d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ffe4c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f598ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51798a80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74056902 +17-08-2023 17:39:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc56a90 +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@130ba775 +17-08-2023 17:39:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3707217359314889721 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e3b121e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512937a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17fe56fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59007b78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ffd39a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17f18bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72dd6e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285c10fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e8a2c4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@553ef139 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c89238b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a3929 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bed95a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f818de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c68e111 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@696e54c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e5994c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671e413a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c8544a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@756eb81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@557ba800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40f92404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76efd0f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d53749 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e66ac3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47b668f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd67897 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2056715b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5baa061e +17-08-2023 17:39:01 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f2f3c4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5341fbfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@400e9464 +17-08-2023 17:39:01 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b1ff08d +17-08-2023 17:39:01 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0, existing:[] +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d0478fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4146eeab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75dfef53 +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 - service started +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3707217359314889721, topicPartition: test-0 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4170f2fd +17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f8da1a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9483b49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e7e844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@454615df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5754145b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f8eef8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@231791f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568a80a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@210ba0e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4911f1fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56964108 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@549b74cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@286bc21f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e3340f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a49c239 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57beaa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7376ab22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736614a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@432a339f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19debff8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f37385a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bbf3ec7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@791b5ee4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15561f40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73451058 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@611e51f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d1cf48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6758744d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cc46f2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc6209e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a111a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47fffb84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@585d414a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4601480f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d467577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@155d8880 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2738cc70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b35b9a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c69474 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c260bc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d9c0d7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ba2019e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8645aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56420b15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b381b4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@602d5f44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@385eedfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e024343 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65fbc5b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f267130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20584037 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cac1724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7741ad60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b32fd49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490949e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@653072d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f60e9b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@238497a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20418c70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4681cc48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e34c800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@598a35b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@758410fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d2f7189 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4523cae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d0c5045 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@185aa2e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ebf3b5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@590ae83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a31580c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67b1ba01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40732846 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4017727d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6046a34b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a4d7cf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a554e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@285a5074 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d53e2cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c98ab69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6563d671 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fc0f359 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de72ff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@247d00c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd68a62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4794e0fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@348ff73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fff94c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@249192cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e23597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15f91f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337f6401 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd09f4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4b5f2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61b99afc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29aeb5a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@361b4d1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@223dfef0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7efe31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a169fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f3ea523 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b9064a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a4ee15a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f89f6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18a52fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2214e1a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fad808f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49b86ff2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78d1e134 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dcd3659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e577897 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@441a5950 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33b5c8a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@275e4f77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b808cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fe1b15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59931bd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce3d266 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0f6d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9deb06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b91183a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797387cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf299e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f284a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a410769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f69aa0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57dc35a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a46545a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@280e7234 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a25a4c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c62b75c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@242ec60e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f6a8f6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68fe1fc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71bf9ddd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f32b52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@849c9a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10d19f31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d8da983 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@271ea8db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@615c4167 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d68626a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d590df5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f5324f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c070101 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62c243d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2731ccd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6183304e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e82a76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c46d99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3a70b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24c32a3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60bc0f49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36421e62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f267ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d200f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49bb18a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228c1e75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@86ac1fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@655ac5a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c143f67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e29f126 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2012b237 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d63c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ea4c675 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f35a646 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a03289d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@281448f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aba4ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123e252a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eaa030 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb31b03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56e25a46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@351632c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6129ef77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72518b52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3625df34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a1432f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cdd405b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62285389 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aecec87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@426c70ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34452f40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1451865a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@944c883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@280f2365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@352d260b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5605a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@279c05b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53bccf7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f7da678 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@121bd920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e2ca820 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b5e6b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9724ef5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45ec4422 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e3a8917 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66a0ff3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d3e10fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad4962d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@708cce51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11b4ae5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c00f566 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@621b7d35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7473d961 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40815ffd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bb1394e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627a05d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8287027 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@421cb868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1035822c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22531137 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69b1f412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec9deb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@725101a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48317c77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec61007 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7a5715 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54dc0760 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@174d1b48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@915778e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ab4b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41f8a594 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1071b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c1bc8f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33c5414d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371596a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ab5bc4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@482bb1ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51fd4edb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ac642f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d36d825 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78050258 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43b51015 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b1c3b67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b03bd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30f7e25d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b2969ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134b6ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60bb6bab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd198eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f27eae6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b00b823 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60dd66db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@328bceb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@439806b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d076f42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16673987 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@506e0044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5340316d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@751b2938 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a1f9d1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a53da3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284a4744 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@434e99c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22657b72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4603f70e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c89f8ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fd28ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1665d88a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f07e6cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c3a2152 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371455fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f8d7a1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f4cd4ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12114ab0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73f668d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630691da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a767004 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622ae130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eddf8f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490232b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd9dc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@352bdf84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c09b9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c0f8633 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b05bfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8c6d90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd0111d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b8e3a4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4993e175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310be53a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8842ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@209e698e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d0aca1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189cb574 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30ab9322 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@632a4ea0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@106c43c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da663ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@178cfb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3096de78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4296f9e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf35914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b315834 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d523834 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@287ecbc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec31c24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f621130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bd1570 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc3c83a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd7db9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3453aa41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f9e2f2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c6fa8ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7981bbe3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50291730 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed0b544 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f19d431 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b332c92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@433b2053 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@80ad7e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69532289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bc6c48d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e5501e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f8e0c38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f392c2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24671ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19b8b41a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64884d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5046b046 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61be72b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dfa12fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f5627d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a776cfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@84ebeeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@654e00da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2be8d304 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ad94a09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779d145f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@624092a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f5d1d19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1497f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50345da8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368730aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d546b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a15b82a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48be7bdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47331000 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c281ea6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b6b25e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10b0c3c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f5c62c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33fd98a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@414bf714 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5773922c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b959044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24721731 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a1fb7fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7136b061 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@267619b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@120949d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55eeac39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@119fbb5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b2835c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2d05a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f02e510 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13abecfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1025b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3727605b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe529ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1365e2d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12d94889 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ef0d731 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68ad065c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54622adf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c27bdda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14f6ba30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e9c114 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51983584 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e08b00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41669a16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5d024f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b7920b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fa51251 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae68399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@695aa5f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2be45ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c94e451 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c04811 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e09109c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778360d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c7eb41a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cbef344 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8581bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@266cc5e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f7b6b53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d4912b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ba7c77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4239fa05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ac30e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b1aad5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@673931b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6add1680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de10019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46bdfa72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6827f7a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a4c8536 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f58a1ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@369fb65d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6152fdfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f39a1d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3536284c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed05de9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@89beadb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a0edd1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efef458 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52bddb7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a72ec5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@599ded59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@188e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cc2fb7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@af2b26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36eee94f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72c2392a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f03d1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20ffdb1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@611afade +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27284be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab25131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cb5e5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32319816 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2470bd9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c835bfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32e4f9b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cab36d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d96c6b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b3c3444 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e8067e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc8cd4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7033c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a54a996 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77266e40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a596f57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bb6f7a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3431f35e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70a1dff6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11632562 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@df4f326 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36645ac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48446d90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@408419cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a43af3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee562c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a6a864 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@750c817e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241da9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57e7735d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7722044d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66b46ca6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aadffe0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63c0473 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61db6895 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24cff954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@211850d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e1c71e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d4490b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@261ac312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45e799c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c777f56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ef9747 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@251dccbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4d705a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b77294a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@420bdecc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@938ba22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e40d4ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6caf8704 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@704ec9e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6b8462 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@434583c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14b36819 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12b4babd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@721c641d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@544f37da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b336eea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce54663 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dc8fb6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5014e2ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1010923c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56801803 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca98a36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5d36fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f48d6de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@595e9734 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79865e13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b1e5536 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42a98cad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fe354fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24def9d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d90bf9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53529c61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e35e7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75a0331b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d142e5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a620e0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56cebd83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@220264aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee8e84e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@540099f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7701045e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46970920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f543324 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f4ba06f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@97682c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eb7bf17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70f47ef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@521ee05f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69732438 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a7f6106 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79afa971 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c96b203 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c341f4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d923c9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79f2e67e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@109d45e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cdffec0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c5471b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eea8e6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64a23252 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8f9f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dcf8b81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@171ccf85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d95f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@223e5d0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38c4949 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@721dce5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a101a60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a16ea13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb12c19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c2502b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5b87a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a0b663 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7293918a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17b5d4fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66227f6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a4de62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d550e5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24fe0300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d2f9d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d07a45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31c2ab64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f0585f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23aa2953 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24af7d31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@926b4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6747e9a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15d5b4ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4acc05d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5917f7f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1452fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@398d8bfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1345eb88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@750b3e7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1b572c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@81cf50a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b9562a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@186d761f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f48e009 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7093e24f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e65f271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dcba02c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255b4f34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab5404a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b62d517 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b88c297 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34d5163d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5402a7fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29351a07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2424f9cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@385c778b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79a01232 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c4da23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7badb66e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@201676bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@595ec10d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7951dc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@119dd59d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2adea9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d54156e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2005e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d4e42c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10dab6e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d283ef2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d59b21f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@128e9b00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57a64215 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ba529bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bbcab9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d0c7d70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@585f262e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a951459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@444ba0dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af2a2d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bbd89fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69d8750c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@700c65ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe7ed10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2842696b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@218097a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@112a15af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e5f5f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad7c791 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28464068 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6980bfa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17b4a528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@438b1300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59545e19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35c5973e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ae94b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c62e9b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67bdd8b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed8cf18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ea85139 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258c3a18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c7d2e77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@146594ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78282ec3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c308724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797b1843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4181396f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78e659aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4057f835 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@614f597c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294d5e7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@632c79f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cc5b3f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75a8dac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44a7b5ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4444af2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752fc66c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26cb1288 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fb8698e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee49088 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35f33d9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b5226fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57bac207 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4078a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6173b353 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bcc3c71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ae76bf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74acc139 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8390dd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@524c9716 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5375e669 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c25037 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@155b276 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3797616d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bad8f58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183d2df2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3457fa81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3847696f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb7e670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e941ac9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@659c7668 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@661367a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b79971 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8bb913c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e129975 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7341f95d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9479a55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d21d830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47623bd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b95b970 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e350965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d86a0cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@becbc10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da4da24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bfc86d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5394f998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3df6d3c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b84ba6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59adb783 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d81dd17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e25382 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6243c983 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56667be7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752c7d59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41664e3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d159843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c96545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e191db0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d7da5e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3bf584 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fcb4e0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39b930df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@693b80e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1adfdf93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f8161d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f4e9932 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a471c96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e006f0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2afb32d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5187a9b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a6acb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e03f1f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d1d1f6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11bbe345 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779e4ea6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6603e595 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7370684f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d1f0caf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f18ed4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4127fb12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e5d169 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74dac98e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40697eef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee8f29b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@211912b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad39a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b8a13e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@713de698 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1858c6ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72529c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fae620e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21395a45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f0aad57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19dfd559 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ad7a7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eadf9fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e78b6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7375e1b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4db50b29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c86797a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34121552 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6836121a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d3fb076 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225d4502 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6416c9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e3f088e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef3af3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ce9ab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be63f6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dbb21e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e02e538 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c8bcb64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@364c8ccf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22ec37f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2e2db8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79cc1f52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b34ce50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68a47f39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52dddc78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bf26ffe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b96e9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@daf8cc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@312daf16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6978ab52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6438f57d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15809489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4909cc30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6587dc78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46374ea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5782ae3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ed80b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b52f2ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@776caf02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35dd34a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@909f296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15d1a5fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7571cfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b734090 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d28942a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce0409 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ac158b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b77e8b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2227ba38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@573c1c39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6708dab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@237edcc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9744ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b7d5a95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a01fc26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcc8545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797a2e42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18b27a0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d7755f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@268a19b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e2a2b1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c372f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c4f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74ad8129 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b002e62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@354acd48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ccfa65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4df09d3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a9c9de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ea78ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c7a5c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f97d75d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d91648 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1233975 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a414510 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2d563a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55d86241 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4da8011e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dc1a47e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7cb77a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aa01ade +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770d4465 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37affcae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee15a46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44edd1ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59de8b3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11dc5623 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255b1d34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f91d890 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@223b1503 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51252229 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c6a3784 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d3e3fc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57d7a468 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f1bf0d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b024a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eacc901 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b2d858 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@145f2167 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47998a61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f15b752 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d17caf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@302e9c2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3befc2ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39425dc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a614050 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6a68cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b560059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7051fabc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e25726 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d8367fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bce6df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79c768ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2e5bd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d6469db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c8ffab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e4f6b91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27067b6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3932f4e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a4bf33c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f3d1007 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b74c997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45cf95db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d85c4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1772d979 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a878271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63dfa5b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a7a7b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@201f0be9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ed9191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@588176e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4407f84c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c9fe366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36d4987d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6531ee07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0faf75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d68fb01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fed83a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c6198d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66d16a47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cdb256f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2845de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a1725c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779e990 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c444d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f17f777 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c7dc04f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4808539 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@644e07c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c2e879 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@669f2716 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6636cfb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7252f4f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21805beb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb86f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e23747b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b33b873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284ff5fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f09d11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2005ab78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36e36f2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61d48417 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@309d8529 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d4eb629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10790c9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5548c6b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@920a5f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4117a506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d24c54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a3c34d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63f276f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a7c9d3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7147decc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab6fe80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65cda6e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b467a21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b87a3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1957c0d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11a5bf88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@597a48ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee49859 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9929c07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35773b42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e412033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346d41f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@298281dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25230285 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12d4feaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c685fd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d9bbe3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd3a877 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79c72ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@775f70c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4be5db6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aa44275 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5497ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56011d79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33da927c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b55c998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7542c6bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b76e7bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd8d9df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6964d0d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@af0b45c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1627f1a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c3bc498 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53965db2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ccb51c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75481112 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37950427 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@560fa4aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@504cfab2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65b9b128 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a123584 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5113e498 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66bd1d62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e25d06c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25706239 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b89752b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65513335 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5172943 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23d0f272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@350fb0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f74b8ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5315d956 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f2f224 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@737eef5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3906ab12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44241a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2427b37f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2027b4b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c75c59e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4865566f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5140a857 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5f1ce0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e0479c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb08841 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78669f90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c6a2460 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21cd9e54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@997e8ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45c17c11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589519ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e71395 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23034b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d6c9a88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc6bcb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30906f2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9f20113 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43727f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c835ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b12ce9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f1ae73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19baf4d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b667b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5d092b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7250b2c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c028463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcaffb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a4ba537 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dfaf421 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35734a61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dec3424 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab948b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6028ea39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f27941f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3316d4cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530aeeca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d4bfb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a04a59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21576ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66db4dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5715a29e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52e2d8a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be95637 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1049d6c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab29707 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed15d16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f38ee15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4119c4f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fed697d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b70aa7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4360a746 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd1e5a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18a3dbce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e89d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@400254c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd15cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c6d3de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77e482a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@259264c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eea06ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d0174ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e233848 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1821c505 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e37cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b3ce8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@476559a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51fab5ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c611d9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2248652a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f8dc8db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bde5b5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f172a5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e976fa6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ffdb9bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76c02f91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b386e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@158c2b0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@379e5840 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d54585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c0d5ff9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68adcb24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d34d229 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a74367 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31a32efd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37d12706 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cff1e4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@413ececc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6942c4d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@571537e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@135c4622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2423b22b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b38863b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25ec48ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54f06703 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dfb255c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72ed7f86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3580c309 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@621f62c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@567a6a1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f723e19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c592475 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65497871 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e6dc55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ebaa4a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf54588 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d12e77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73835aec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18775fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@398f802d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1581b3fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cbccf9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e94758d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5f189a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d95e57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@190399a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64c6af10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bef0da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4edc2d9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b97637e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a78fb33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a30e086 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45af9091 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d387c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68af5323 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@576ba60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c649947 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a0accd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@784c311 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3711cc3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f27d53f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f71d2d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e4c1266 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27057580 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@369ad272 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c7131 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79ed8c9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75485f54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a11acd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@710034b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@649ccf61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53199e59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ef38185 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea69156 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c43a624 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b70ca3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719a4ef9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b3c4dbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aefa678 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41266b92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@626808c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b20e09c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cdb45e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed3df14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c5fa447 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70878150 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76751ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@466e5476 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37e2ad60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ba327cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@257e3dec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@110700d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33ac263c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d3f2cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5162440 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3ebdbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ef1f3c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30fa0379 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fa0502 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d70a343 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c6e14e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cef2bd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1869f55d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405b37a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c4eb21f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294a93d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e5e58bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a1c6af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e165e0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733f881d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f288d41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16d5a8b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a8f624f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25ff6863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e86bb18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25ad387d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74e1bb9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a036912 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f65931 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dd4e7a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@341080d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ea24bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3666cae2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c65c27b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c43ef11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38d8f1fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38691dd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a6d66b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3925238e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@468d4271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36fa18ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17039808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7aaf035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b18b47b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d41e6cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4689e4b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16728f08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a75ad1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a12a48f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33375b3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31bf989c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@738c2da8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c95d98a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f29f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c496cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ab71911 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@789cd844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3953e6ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51d65fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37508bdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29102b8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fd2dcd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@610e6a40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b30bdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a572e71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637a41be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71347bfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b936bf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7874ee54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49b23755 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8b763b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be74a58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64be053e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6e9e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c941d7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d202d3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69234054 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21764695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd5f805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e05b71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72328af9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e328c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33a4f12d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@301c0ffb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@569f2e43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec2995c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a7b2830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ac88b53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450a5d7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f3824fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c61b984 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32dd3b48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252b79c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8a9e6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a36b907 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ead472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f710ab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60c7b0f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f06d36a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5848c5bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcb48b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225617ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b221d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55d954f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70720830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@628b82ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@391a39de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@267b9cc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@462833b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@780a557e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10b64768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a515430 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d637754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e86e23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65c7fcb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d7d4ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73f8cad4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77e52bfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d9d63c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16d6ba25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23cd6c41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da9091d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b6cce34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18484cee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50af94c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65fb8e2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383bf1ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62cef51b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e6b7404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77583134 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108a0b90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f942b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a7dd58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c08fe9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6656d62a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2975ccad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79689c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@250bf11a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@753c9417 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@722b3eb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41404183 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bb9eb90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a345cdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14171d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@424e7427 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@365fe295 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e058a3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34408bf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e2e11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9c3f6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e5a18ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e750798 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a6fd2e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a5f196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3934a010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f3b2969 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b2704b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3167f06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fbe7b54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@396869ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac9d09f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a17728f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@469fc842 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c40c40b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6859a634 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@794b22ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ae3809 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf4dcab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e39ade7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ec70a68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e190bc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13559725 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@90563d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c2980d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4003945f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f50dfc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d734fac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f23b2ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@735e7a28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a093d71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de01db3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f66feba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe95044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c481c7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a887c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d33463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c2d3c6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505a6261 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f9793f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@658939cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22bf47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3688c48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d762d9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d3ebb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14d06148 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@531db556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1b505b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@279bd7e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3056387f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e5f9c26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b538c3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@620e444e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@639a0db0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11d60349 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e34f31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f5597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359b3f61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14f6aed0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1860f1b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64efecce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8e3b705 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c218e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3630d156 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da2d2c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ec03ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1503136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31daea2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a551a10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527e9bd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5227c92d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@344aeaf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16edf3f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30eb62c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c69bb59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f17c74e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@461cd934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d0a0e1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca8db96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d0456fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d71fd17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f67242 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a7861a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d44792 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175f8b7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e89fb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc48be4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0bb719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5164ef79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5171e7f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68d3dd92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f071cee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@796e2550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27fe1bf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51377dd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e0fb75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eac165c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39daa9eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e982f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32b3f0af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca30e53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50da7ddd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbbab10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62a00cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d55751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4562fec3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57932e54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35b7fe6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59344852 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b722151 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45dd4fe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f13ad9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@341e1af2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efcdd89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f11dd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53dc0f0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1751560e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f89c1f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@569f6860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6508a1d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78f46a50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6963e2b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44125b93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64342ca1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f497c9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49495799 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fd153e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ac5259f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e1c17ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dea780b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38316005 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bab4341 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c79f9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b93950e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4891e331 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b11029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b347f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76f8508 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@402066bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@114116ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb767f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@108bf843 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@535d6173 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a33da94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aebec8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@556d5673 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d83f354 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3930730a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c7a6f25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@439b7017 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31703263 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@be2af64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@244efa7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ccd2d2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c1ef3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1219f494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d74187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a6a0dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5c20a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@323cd48c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@491c7035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@681d898a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664ac536 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3206e5c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18b1b66a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c642d2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c3738e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299ac87c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8d39c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd7628 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14d8aec7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b7e5d08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee83285 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@548e1f46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f13d86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cfeb62d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637dc32d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47ad942b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b3c9c55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@221a04c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ff3cf04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4541a1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b25f0f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ebd55f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6851ba89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c71c16c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48573a54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f115f2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a7121b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6849ef36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@193dedf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c1039c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f4fd77b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58d8a7fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7c9b74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73ff416f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@728aa464 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff032b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@251db300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ce06af5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b6e5289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65659cef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da8b6f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0d672 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb336b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f84fca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a0f6053 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e883e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cecbdce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9a2386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d36cfbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7250b530 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0b7479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3690e48d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0096de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a0e477b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad650ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd365a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@328cae66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad102d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258f0de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e65b74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@154e22d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475b6d30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c0bb05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720d7dd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16b199e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a1f44ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d00de3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147ac1a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9e2c5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed61172 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75acd9b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37196165 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489a5f0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6899aed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15fbbadc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6995f34f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73edbb47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39790925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711c798c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3755f36b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29fc4145 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583d457e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a7f87ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5229fb63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19946e0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b30cef3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d467c9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a97eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af7b497 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@179da26c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39918e8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72060bc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d25a1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fa60808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a760a97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@786c662d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f09d186 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20d07c27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a78e818 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@313eae74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60eadc34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4378b54a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e2dad15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a24ebaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9991a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5acdbd9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19cbf6d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54ff37fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fb8863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73edcc0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d6d1cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183b7865 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4391 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d8b5a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e5088fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5deb3301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736e4e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@402a8d3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a27eabc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ed0a9c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e71772a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a264cae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f313b5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44f41fc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d9f852 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69d293fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fafbe46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b41ac65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7534cc37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@804d421 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a5b16e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cefa143 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe9839d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e2f39d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@335c5d16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52d81787 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512e932a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27a6ec89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7643b186 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6435f4e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41278c89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f5f280 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a51a6b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7108e7d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb66ebd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72a9ec59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45faf8db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73fab9da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c06e29e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71701463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635e06b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@454c576e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea55f4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f8797f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c7f4300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7baf88c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f64cfff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c2fedf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a6eca32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d009c01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7452a769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1771b45a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b02c3dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@625bac29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35326e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@199a91b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@544bad7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bce6938 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@713006a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cea1e24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35437488 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ddc4cb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ca842b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30df7c38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e96b99f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5160c738 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f438a59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4361135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc71a6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c941aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0b1da8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f800e45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8837baa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a75bf01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@496013ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@560dcef5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45c0a1b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@525a93bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1beea789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@232cb867 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a168be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e0f83a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b5cc9e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d75a76c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2579c2e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@473acf59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ac41d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aea8164 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@216f143f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30be574c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a1ccc69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24638bef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df4bdb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fb29175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63117d6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b95be10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d481ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c691d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d473a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c72096 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@113b0458 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@def17ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e452c83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6189724d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2859a3e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dfc09fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a9a17f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30ab15f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5588f6a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68869e3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22692956 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a058441 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582a4e5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4817c2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc9d670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9886044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3da9795 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c35e3c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2deece65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67ae11dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@309f4483 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1317a29e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4851cb7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f661358 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1517ff6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3abc05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df6922e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1265b21b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38a3fb9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5617b21e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb93d10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2073b196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72bf920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c472c99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a40cdbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7823463 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fb82f4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@acef3c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36dfe208 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e8f99c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3450b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4399e441 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a1f8198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ddcbd5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7171e1a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d46a822 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ba3f553 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39d9f34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9f1a0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d0ba26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214dc8a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e469780 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b0cb325 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b3d31e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347aeef5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4efa856d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6041820c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72614d9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@118f1862 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc426d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67f98445 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f0c3082 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76297709 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c7135a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778159c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@977ef94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34db508c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6152a78b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce09d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2addda49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ecb0ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30f62947 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28e6caba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c2d527d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d65e696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c58526c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f5f247 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54cdfcde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@367ee187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383b1e16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45a1a49c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e318bfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f823f62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f295d2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e49717 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f8a04a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4baffab8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e7a3eea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d15461d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4f4105 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7901182e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7603e48a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31e161f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61159a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7720f12d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6e7cf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@243a286a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@286ae155 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5833c4ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355782d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40db9561 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f44b466 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@799613c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70de1767 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c32f480 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@764f8ae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f8eafc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f6bf271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@448697e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e942d36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ae3bc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6436a95d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147f5be4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b6d29c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee8754d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fc5598c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@310b99eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@233a8523 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eb5ce14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b49f377 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7660e534 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1046546 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4101ce1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15074f00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@643f41d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46333645 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@98d61b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26dd40cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3872ac0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672c3459 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54e41e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d9a2ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64667541 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@626e1b68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@155837a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11108bc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@460b28dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54acce2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d162d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c6db6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a9444cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@387b3e00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503f7b5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7381d79e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1b7739 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af1c964 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@113fb0fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c7fa863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c5db86a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5853f66c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34071dc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25aa8790 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2435f5a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631287db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f77777f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e90eb06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1583bdbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d8dc8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c08321 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f919624 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2777a025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f14cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3184460a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1287d6a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@86b2a55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73d98ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@607e328e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c608abf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d8828e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2219437a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33e2eff5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b1fae44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278c76cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f66f3d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bdf208d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6db4ce3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294ad76f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ca51fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4383d802 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@88d3b60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778374b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74459e1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@295b0b5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ec732c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24a6a07c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ccb7f98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dbb8ab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e5fd3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37510cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c19213 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@417e4487 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48fb4d9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f39fa6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bbeda41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5857e1cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ad2e8c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b7409a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cc6fa3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b782de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63102e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d192129 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160527b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0817a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a00aef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56e473fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6963de0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@251a699d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27efe8a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52e13c0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258503fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5325c057 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd79060 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@508ddc58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d9a9769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c0d3f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1723031d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@297272bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@205a8925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a3a9a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9976982 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33ca0597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@472db16a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c58ee86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eb147f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d34110a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27385727 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c8fb198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e5cf08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75f4295f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1010c9a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5152b942 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2739a1ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7353da86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@659808c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21c4259c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b98ce92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26f2aa8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30948bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e17587 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e80d737 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d15b2a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74904905 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c55575d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74640e8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64747259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12dad7b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e3950a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a7fb6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5706a598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13258784 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b360b2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6465fda5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62dbed4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@390b0032 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@429f1b4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@95c5e6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62ec8a8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3870c72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f83a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@523cbc43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5402fc6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5d8e05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@170cf08a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e5988c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6347c95f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60695e94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10116ab3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cbcb1c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@416c9e3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b5bf57b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70845e51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9cffb1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c771966 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18b3c1be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a97622f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf66f73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28defda2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee66736 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f91fedf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50939fa9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fff4bf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123d42ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@420a2174 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ea8904b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bfdca81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32ffd06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@103d6d96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5add9d12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe1fd0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33daee3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48f0640d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e73550c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ebaee65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5af54489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11ee83d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75516fd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c716edb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cd0a0d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9457eaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7794bfaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60840035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93bfad5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b294d5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7992de7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3957fcd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f6c4479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ece8f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6140e9fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6777185e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab5d745 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@107b2116 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@769225fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29f8769a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6db3931f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24896c2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43d31094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29f5f2d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58afb798 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a848bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c24b96e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ac1b55a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff7cced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aef881c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7349a9e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14c5a9b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65248f61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23cd54b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1df5dfb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24c46d90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@655ab1ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cfb1928 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fc07367 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@186f35e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@288a22c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4c4ca0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@400b4705 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21aabac3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@584f1014 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66d86ef1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c713e40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f9ffaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28968114 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d9cdd11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@786a61dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fa9edf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@232b4539 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48b47018 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63fb9c53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13cd73f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a36ee3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e967d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4f6ab9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46e5cc73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b7f9594 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5162bed7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143d1ce0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c53dc40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b67356b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f37b4d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@239179cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c21022 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63aca91c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1df30bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29286655 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78ba2052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e589210 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a3f88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c0afbdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@459beebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f692c0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b8ca14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d710c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@923ea17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9cbe61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@511a2bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ae0503b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c857e22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b0aa65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f5a214 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eec4d79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9df5ee8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8f3e34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21f1ce73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d5ac8b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e81ffec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d3c0096 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1211d314 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d59ea7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@385a76ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65bfe9c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@792a1f13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cef9637 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77696e42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d02709d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44f928a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2447502f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18e0219b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66183c82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b29ec88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75711fea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c840426 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bdfd918 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f362147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1245ed04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eac6e5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b8e7b8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356c43a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8fca5c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359ff41e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d558b0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a738468 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17cce44a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d39cda9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3520e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a03bfc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a1de27d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59da2a12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c1d7ba8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc7344f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eec2140 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26ee0435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6169fb7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53bede55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c970343 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a0e6348 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6492fb51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41fbcde0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65d25387 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cff7856 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f4458eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304f042e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477637cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606cdabb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de2455d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@254442bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53fae1b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55c55071 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af22f39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da4e146 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29d74a1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566ec18f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5ea79c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67fac5a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@746ef4e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@542c3152 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c560e0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7db005e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7efa6f00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4be9672c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37b33633 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a51a23b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@677e561e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672e8dda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49023da2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e051ec3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b5e67bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@88d7957 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50baaac9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ef551b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6497c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dc01298 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a9030b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7462c688 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e828541 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@332b1adb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aaf6ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@611e2695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b6c1f7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50fa270e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b3b679 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@148912f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f4548d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e06d461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@579fb3a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bdd1513 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ebbdf70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e6ab1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c4e38e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd8c7f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f28c320 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c350ba4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f1bbe1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@180f2b4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3117caae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@551b9f03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@593918f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388d1dd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37c0e2d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e2a80ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fc5f114 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ff61ab3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1099c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25556751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25187fbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6af4da2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21e1802b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f0004a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49037b78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56aedce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a7d590 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72db961 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55a7ca47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f149a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11ec079b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be04276 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10fe4abf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4fad49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c6eb1a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e4bdad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772da61c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e86855 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e12089d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e2bc527 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@242c89b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7052f7a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1a4fdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ef8bdc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5984e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2951d53c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1589137e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ed48a84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c283b72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf7edfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26b8031d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc23de4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ac6ea5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@570f373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b6a82b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed684a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cad0e9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e448f10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58e58903 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@420cd51f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26b1e225 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca7d5ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@787ee8ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cfe6ed7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75f70fa7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47cdac40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@487b35eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675c62e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a9bc39e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9ed521 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11db6cb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15947f20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6167d93f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a62fef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d525f63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b094696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3478c4e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12875cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5de1a0aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cfd6076 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed0bc8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26197024 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ad70bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c440ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a7ba4a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78e719d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a3bd149 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ba20f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ed3e1f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a25772b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e504cbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56655a25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2748d555 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@166562b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8cf17f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c43e79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5747b789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a8a7f7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a07c5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5000b527 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a11ab51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3597acf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475fa464 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12238825 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25dff213 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d987b65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@575e9e36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b25658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3276817 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4a7845 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e46fcf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450d7bf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d52d494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a4132b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3abcfef7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14c0c0b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f5d50c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc3394a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1553f63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1302e2f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc6301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@438003fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b0682cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f17b875 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aa60afd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@497a4f89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1635ac93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44336a30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ef861a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65657ff8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e8f16f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b285f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c7a494e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72ae7b86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5430d3ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b80195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe600d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e153ce4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6a272b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5f359a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62a0624c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@655590e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@482ba01e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cdcaf64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d8f06b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64144170 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71aa36bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58240062 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dcb2f59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b12e051 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bad2059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13720a5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47175e8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bdb3e55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f67a9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a55b305 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67cb7def +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2520ced5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f287a6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38523d17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44a1bf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27bf2bcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b88de4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@429becaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@165ac2a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e572ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65b2af03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7851d98e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@385e0bae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bcf7452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b75029f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d9e33d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15d540b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3598cbbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@367a566a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad6c3f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b9d0da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568f2afa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3222ef47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e4763b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75134fa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae47300 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f99e7cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a66416c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35677bdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce27202 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c754bd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14701816 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac720d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@456a23ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1317e375 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b3f9a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24d1a819 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30b09a3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c10cb16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f6078c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4ee66b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@191bf896 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9d8d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3737754a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d90b079 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b093a15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664b38b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a5a9c12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@221fa58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@601ab19b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2bbbb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@612f064f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cd1d308 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59723160 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b86c7e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4202bfa0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20296d7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6898a12f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ccda12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52648ab4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@394d6736 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147138e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778e6c3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1496950 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53efca98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ec4167c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aaf640f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e697e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2df0366d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43e9b259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72d72937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@463ae101 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad5f4f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64a0e88c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f75fbbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed635b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be11ccb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18e67fb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@184ecf1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17ba525e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64ce1326 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f7ee3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a6d301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@644b5dc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70083545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4851f114 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f780dc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6163a6f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50701583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17ee87c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8e7aedc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74cff529 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24433a3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477039b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7254eec5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b92b1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44967cd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed2802 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@579acc5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67cbd447 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10cc04da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0a83f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a7348e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43a3c20d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33b9e477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c03bcb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cd793d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b985ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ddf57f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb896d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c31c790 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71caaf04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@560d0e01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e1b3c30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c846f24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7812593f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d12d3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513c736c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ebedb20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@448fb469 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b78f93f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ed4c2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61d2f7e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c72b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14ba9060 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47653aec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31fef056 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50027034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@107bd3c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e27397d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@389d1aaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b5d4e58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35765704 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@725d7c06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22c55880 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337bed0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b25212 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d96a5ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@251fe9a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49e06394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f649a78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c4fa8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4c6f1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@677a45e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c86a429 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cc633b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@323cb789 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dc914f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e62da71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5374119a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a36482 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6716dbb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6faf2471 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca400df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33cf0bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52a66031 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44bc4ab8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78220808 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c7fb6a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7db9faff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b76967a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33fcb754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52d7957c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@600bf3db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f216ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59313c41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a33e6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@889966a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ccb8dc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc86b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72366e94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36489c7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66bdbaf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b3240a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3830b766 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ae97772 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@410836fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21fbc9a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ccdd17f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d9e6773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da11df8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6268792c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@105c0a37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e1742b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bccde57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4acc7b1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab1cdc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e770ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73c44d1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5af9f4dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e92d5b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28f4fa92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7909ee3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc3bad4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1484d5a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bab9dc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d65e086 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c01fae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1a1d11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@291898c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfb3cb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78fe35f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46324d68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b812bc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@596d8ca7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50357bd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43825b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2645999a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13783141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68681321 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db89606 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27bb4362 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@751b8ced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc26874 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34a3cd6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49fe38c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@602bb9ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee4fc46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4069c5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c06d462 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1df765c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8d0bced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38041de8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32ab7b4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@483c86d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45867cef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d75c2ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346db301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@184bb153 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e5cd7b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dcd2a02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44617701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e587f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@98c2925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de58f2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f336ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f84d578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772902ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26646f8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d8b69ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@341d4232 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b9e1a7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e832cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5891e09e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@643356c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d758a39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a0497ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d461f04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2231b595 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7344d7a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db0a4cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30832240 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb1b4d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a1dd911 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72338a53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b0b2777 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@498e4fba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dffd2cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f0ac35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c203a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8a6fbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29709bae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f670bb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea9dae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b7a511 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ab7e9a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d1a4fc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d94ce72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77e16ffc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@325e625b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d857fa9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3659d735 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4abbd08d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b363f3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2de871c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5570fb34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415b66af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b2e5658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a65697 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475db64c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d29de16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54bccf7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73092993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59cc1e68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a86a229 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78735b23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@420cb00f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0ac978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@462abda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66aa4236 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41810a24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c6d5a65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d401049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49cf5625 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d6a512 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e791f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bd1f3c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489cece2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d573ed6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@80406e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3fa3ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ebab53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@170e8e76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f15a963 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb57cb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@300ae90e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5314bc47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41d849f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@549dadc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33387ccc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45b81868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ac66d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe67029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f48ad05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a369fdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f89baed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6322f00d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c859e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@173efb73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@758265ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@154518e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab8475c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a3ddc1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf45fca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f8fbbfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672597f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c420a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dd847bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fba7d9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346bf786 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55066f13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6c37a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7696efff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d2ae16c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb27b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4058d284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f02467 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b1b84b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2398e5e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d2dfec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2581042d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a81a0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56fb061c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e58e90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48685e6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a624d1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34a777a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dfffea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ee5c47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f4cfd10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50d9d36f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52512531 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57d84758 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f4484ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711870e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2abbbb5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f4cdbd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74e54a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4648ab18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31de81df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68156642 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@367ea43c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35edc980 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70f8669b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@331c29b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@700034ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfa52e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@180525b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a86dea1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b520e36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c636fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73dff3f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f5058f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2458d497 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@575a6736 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b96061f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@560c6a68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3492f052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30f54221 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bb81cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ed272eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d611f03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@571fb8c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5a1015 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3162665f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74beee77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e4db1bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@428bc8d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60791e48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b78966 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c414fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c95cb35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b617b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b954ef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b06d68b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292860bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d77f1b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c04926e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@114bb3eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0c14f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3934a061 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@716bc2cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3df3f3e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2643f303 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a686802 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@153a881e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f4259c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e16038a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77474232 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36fc5ebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2032453a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0618e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@539d9fc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74032b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35e9bcf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b065b9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@245a9c89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2289cf24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a7ff1a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@730ca3f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c2d7c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fded87a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a89c6c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c0cc76d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49e52fba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@506123f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@545a40ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e57c2a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e2fb82a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57084498 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@515333b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14191d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72dc973a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67e5317d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fe368af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2221a392 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34d295ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7af22495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ee09d74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61728a09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ad3441 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a90766d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60514576 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1586b9b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d21887a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14aba2ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ebb2fbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ed06ada +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f035b22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26f432a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1951737 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cce2b65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29e1a195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e87df16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@702595e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a3adfd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ef33e03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e9c2fa4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc14d80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f9873f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606dace5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f735e43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e5fbd60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252411eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28a39b2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da7493f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e704556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bdeae3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f103684 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68e976e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@247fc456 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a6e119d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b4b042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4749c40a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dcee5c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77c6ed1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b17506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fd18e4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55db8498 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15353d32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289173d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b812b75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127783d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e97e008 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d3b31c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f736997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@313458eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20b27965 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d3d9b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70e83968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6308c49e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5042e0cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ab68d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ba4c049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eadc997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69ae2031 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@677e1ef7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a752cd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a54b571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1263a3a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd8ebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@660d793d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49dda7b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46ac739a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fcd9586 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2284c5d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6401614 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23d595e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f755bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d9ae2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62aff415 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5a3f18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37dacdba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e5bada3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@349fd21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fd7a0af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e810df2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d749534 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5198c34e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53c8c30a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f0aa1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c9442a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b8164cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ec42225 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ee4dfab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@207b038e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fca449 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb67b79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7f6a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f28f2a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a13cfe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36520260 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e68f8e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ede630 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3285138a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e246b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d843b03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4818ae5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e02c0bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18539242 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6473fe68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c2d96a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6422bcb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e7f3fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b1cc136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b84c46a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a327cde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35151a77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48fd7892 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75675273 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c4a148d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@111171d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@320bb289 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d309a71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21820726 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b07653e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5477072d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d6decc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce7657c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cc9dd4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ebff90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a216a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1549590c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4738fd6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68009d63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3599dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15eb28af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15b85cdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793f208d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f22bd0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53c04014 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9a80c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f2db50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e32d404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa3f536 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1663bf06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9cec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cd3b8aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c99f443 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5709eb59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dec88bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c7b7540 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fd3b160 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1798c0e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e481bdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29deb220 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b145eca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a5a1688 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e276e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@187183ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18a7c20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bded4cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cdcd006 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78a660a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a1f78ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67aab9cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d7c09df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2694f794 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@771d23b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@454b8044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48745bc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@495d3d0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57439724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e936428 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2453847a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c678b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff735b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a90ba4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22589d36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@156c6a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6340d7ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c9d6306 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a4af08e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c81d4f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@99a900a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e88476c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530b1ab3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31cf6a6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c08b69e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30673f54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b326569 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71e4ec60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58f10884 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51650151 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79543f39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f055b9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f60d3d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6763ae7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e0360e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5f011b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22d8bfe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ce1856 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@500f5650 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73fbd293 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143b14fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d2b282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b45c73e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7607f98f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a57b778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e755412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfc14d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62fe4ba4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6595dcf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11b4d190 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa535cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a96f654 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1afe18e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3620987a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4558d83c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3801a295 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@465c978a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@748d2473 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77979be6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b78a8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6075a590 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d46bb51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36c547b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48d87035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54c159b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d18ba50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51fcbca0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aae2772 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2b1171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37e9f6fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c4451f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28fa77a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbd4b55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371b0a5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16565d1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dea6f14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4828661f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278efc9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@314d3f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db89e4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36ab0f7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66115f19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43bfa355 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8c1f90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54b21764 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38b7737c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6acda34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c8625ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18cb002d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b64acdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf9a8f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16bd4b3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56b7ebaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@688b87d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e03445e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@221b1800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b98657f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f99eeed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b7c64f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415da083 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4966964a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79b8f70d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@213b670a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d4e9216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70e79133 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5910bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34202d4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d07d7fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a5a961 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ff8d10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3280a2d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41cb0375 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d09448c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16e5d330 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21c46d17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@380d02f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f071c58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@274320db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@256d4df1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75a7d927 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52e5fe17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c1e4968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@235393df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68f60873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@159e563c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd508ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3427af20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c4a8c47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b820844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b3e3a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21e4073b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19748ff4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af64b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@353639b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@346a3df0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9dc0fa6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@702f50ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74500ce1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11302a26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7878b699 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72daa9c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@699847f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1668ef90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b995fba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c2a6fa9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46daa82f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@179fdb0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@761a32cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29125c80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a1a025d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53b4c196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@741f0d77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd2b928 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25304265 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50fd8f24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c8544f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c4adb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2140e46f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d78dbcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bacf616 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48263fb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ee3c941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359ec9ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@398e9509 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@402cc601 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3075a684 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7367f8f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e0c0d41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1836d84f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a1c4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49868c2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17bee65b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39e4b043 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c396b70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11a3652c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71cfc57d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d3505af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a709192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@407007f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f9bc6ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ac6aa8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44745641 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbbe95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15c49e3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a69f4f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c4969a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@151f69af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241b2855 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b0be6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4633e3bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@280fa8c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@552b7413 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@112ceb0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48dbdc8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc71cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13426887 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65408d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736912f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ed3148 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566b6d42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37a1d96b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce4fbbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bcfc19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e309651 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7637a7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cf15836 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78f92585 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c1a6d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e6d8c62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@787b5f71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499f88f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627781e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@341bea24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1639bb01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23544908 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e8e22f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752fa1cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e95eddc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f08ed0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40fef398 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@485723b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49adcc6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a520c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1670655e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@674ba6ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4022d895 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48094b78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc68db2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67823e72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31b2b467 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33e26ba2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa0db71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c271ff9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8928796 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@280d525d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74aa6f7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60666ec5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cd4fb68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e89d115 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@197f8825 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a3ccd89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11e8601d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61191a4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44298ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@283e5d6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a8428fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5366869a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4373d1d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19fc21f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e429f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25513562 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7871bd6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ab873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6332aa37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f646916 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a8b2f42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fe7b140 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f2dd99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75b23b68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1620eba3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@449f2596 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b8e2689 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f833deb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e6fb64b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3618785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39e1530e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fbb5dbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca56fee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43f78450 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70bdc603 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@253e3315 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4203f243 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3da93b57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@233a7f3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774856d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a042396 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e83c602 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e30896a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bfb949c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10681ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509b6b3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a4ad39f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dd2f73e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d8d434e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66441f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4abeee2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60ff8b41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@676f2323 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2197161a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ce54b91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dc7ecf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb433cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ad1c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3105bb75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52cc919c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d59f25a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c0c9ad6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20ce4031 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@163865c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34518e7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c53680c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4e627f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b8d2e3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53975f6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24734e1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69e39d76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b1db2aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@298d4a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ca5614d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49d85382 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd620f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bafe852 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c974f63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17d85b30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f73092 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6ea886 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ba4101f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ef0cd80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e53136c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5611b6b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d2e608d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255a2865 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b497d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@584b4569 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66df6ce5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1232695b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359f7ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37fd0504 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a58ef5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ba133fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6586b9ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0c452b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57435f0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@755f80f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2544ac8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d399011 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30cf039e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@391e6548 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393e8630 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@836301e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40ad1175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ad3080d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47910061 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e9e7e4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5690fe52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@449117ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bcc8516 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6adee34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@579ef2dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b7cb9bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74776bf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d42e704 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d6af66b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43251923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c42c25c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6326e17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c810914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@786f9a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4c8b28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@81b0c40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c367dfe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4df217 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a1277d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75884685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a8e0c47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19609e4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3556c8d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d0c94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58ebd4ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45648b3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4677f325 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ca46ed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aec815f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bb0c255 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a7eedd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dfaccac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73d24435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3d8270 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33e1526c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58156710 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@390773cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bfb4899 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b005b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@216ec519 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@237563d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127b58a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40341291 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a55183 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f46eb60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b0166b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f02696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a6450a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2251a765 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7585ee0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4377a2ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bbe1b36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@447be3ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e54a090 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6048a392 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b347306 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60c9b170 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ba2cdd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e53a10c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a75183 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bc83616 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4649eeb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25716cb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c61ad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7374e37b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac1b7d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54fb78eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5de2e025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d9bfd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ed1f605 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7162e514 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bcbd0d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f289702 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eb8273c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@681439ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dbc0115 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36170727 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79d9571e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c97876 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d1063d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53307969 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6062b5b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b137b04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45c38d4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e85fae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@605297a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60190bf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39694345 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e2b7ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a300b1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@325efd0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@415ef36e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74b7019a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ec5a4c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b36aab2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc5f939 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513280ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1212fdac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c6fa666 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed19ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ba06923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eda1466 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ff7d14d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc354d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d2cb38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b058ec0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ac94754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d8bfd1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e536299 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69678a1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79048382 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bc5ca9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6eb73cd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b55747e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6610f805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13744f71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4472884c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5414e238 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3525e86f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b367d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@547bfe5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278dc3cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d7131c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@608450cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b06eba8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22dcdebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16170c1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d72c21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@850d6b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c3dd030 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62644a5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f3fc415 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4365c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@234ada06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63bc4299 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12522d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32949f57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477bf9e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d2c315 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@456f70cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53173110 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bbb3e2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b219623 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25f66492 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f834e8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d5b7339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4adc663f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c984902 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9ba833a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ffefb8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26ba99d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3874145d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@650ae33a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d505e97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd628fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77980fa6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4539315c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a827b8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4cd21e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@331febe7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f209f09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7637ed4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b183874 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5714b823 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf7ad1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25636040 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe22700 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d455d7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c8b969d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d49c2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c606d51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2be96707 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bd62399 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d25ef33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62201d23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2918f3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c1be1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5f9643 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3874412e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3692d968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b6a8e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57dcf58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f9a83e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f6350ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29041655 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e80b22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9264554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ce7a09a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25d8aded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@276b136f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c5cee33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dae6e52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d790dcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12768b64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6920d26f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1702fde7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32308dbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14bb1d34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b670666 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0a2128 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d610691 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fff0b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e892db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6841f8cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dd13ed8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53551186 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@552d3f7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@324d2297 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16cefdf8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d6c1786 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51060b81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a04c2ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc4df29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@669935f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a260a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b9fd35b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@666f7c96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49f6e31e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d4525e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8952011 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15837659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b9389e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f8d9312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e39bd5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3006257a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47ad4637 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@344b48d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47ebf06d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4647201f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b765a1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5158a73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d657a24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62dde64f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ba2f6a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@124da5b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24cf5982 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292d3cca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@516c8b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a89b5a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2454f039 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@645f4315 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@573f1c27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@602c9fce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63fa7eca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d23de4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52523bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56840c87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@663824b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a6fb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4644b9af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30d275d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e8c2c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c90022 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22a35075 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fcfd20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50d3c914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59920b3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@618ad883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7793c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49acdefd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aee71f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37d76949 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7903b198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56f7d373 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@211640ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d07e8e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49ec13ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21e83dab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32e80213 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20ba9479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e282a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6987e351 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b0d8ce8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fe0caac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720aba69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24c15212 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53dce0ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a80e5d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dfec918 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33014977 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ec0629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a7fd57a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a26d24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68d1dc96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f6b42eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b7b9279 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9f20d7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f2145bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a082eae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2df79bcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3477f2f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@511e6550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a184c83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ba09bf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30cb5e49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@125d89e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac9b379 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4509e0f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f5ad70a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513671a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66534281 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cd710ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@154c78e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779598bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bae9b84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2a22e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29d78ad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@725cbf24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d517850 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e2560e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1676d5b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@144fdc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7519646 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1692f284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@366966a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10e5076e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b0d4092 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a8a97a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67abfe7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f4d4b96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e35893d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60cc14e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582c6eac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d889fc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff68f29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f9df52a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@547faf7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d207c8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26742bc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c0be84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fedd950 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d70826 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@767a2fda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ed9b0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@342befbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d745720 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a57a3ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635b7e0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8e613ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57f49881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3059232f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@84a883f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@308bb39f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4348059f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a711682 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66266a83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bdc3a93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8bbb28a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3acc6019 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6feb67b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589c07d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50f30da4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@966630b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d845b79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7645fe9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1343dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@457112f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b17cd4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ff71af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66d6cf86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a34641a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f6cb7a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4627d3ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16dd9e50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb02c25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75fb67f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dbb6a1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ee4ae2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1951813e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@373ba111 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6186fe5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64ba11d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71f7240f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b2e2009 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d0c5135 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@790cc5ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31934198 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd701cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@747d767d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b41ce8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4075ddfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@234f4560 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c1e11ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@594e9542 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37ba29aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e871e78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225ad303 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f13a68f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698229be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f208e28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b969004 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59237027 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21946d64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a7ef6b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6517434b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27f7b586 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258a1351 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@482caa26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@177eb573 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d3f699f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@785ae3f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550a0b45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@373df9ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b80d53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e5233e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b075d49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33af99ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32fcc809 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4e18a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c72e95d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53dfd120 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7e7d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0124d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31320ac8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cc99b1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24786623 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17ecc1fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36a9fd83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e8992a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@146b2aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7054705f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9e8ce33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b6b27ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f8c03c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c094eee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3df980a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@717993d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e144c77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8ec1d65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@778ccbba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ad62993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668db34a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee29dcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f2246b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64c37e70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5f9641 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c952a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7853ad5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79c6cd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e20ff5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0123bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4edd4ac7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a431d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eda227b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c64e0c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284b08fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad1b080 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e7805b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e2e3ee3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e52db3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299039c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ac91f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f4a6020 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6769e1f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1f92ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b28b9ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477f89cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb19ebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c0ae954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4273ccf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@797934a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ade114b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11951b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ddd5191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d318088 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@178f922c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e7bbea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@98d3324 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7849d75d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13bc348a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60c10ef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3505d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f0d9b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@449ab383 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c44d326 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47760c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347a0cb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@256db71f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f3a9d77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1074a914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@740f786e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c12ecbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47380f6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72b13295 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d72e8a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@562e94a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51eaba7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ab40f4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40e9fd7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62c0ea55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66fd9d22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cb938c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b42a33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c3f652 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cdd7f25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2907016b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b2053ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72c8599a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@158b24b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4967332 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc1ce69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5df55ec2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11174761 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26afb992 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45fe89a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@328adc9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@198da9e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42b5f06e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73379259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17bde6a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@638baf3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed005a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@413aeb7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@394a9d1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7061771 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9f5d719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f82ac50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@724d68c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4547546f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622d0ec1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6847cef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71d5f010 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e4e3669 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41f6a67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f95412 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c726b2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73f0e9b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c43f9ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@140275a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb64f0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fffba84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce02885 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bc8f7c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52bf1384 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5096a8cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d5f065a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1185a8eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b0caa6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e39bd00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b911d4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27ef2c22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79eead79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a2cdef9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79c4fdb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27944094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b9db47a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27ca7ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79bd2cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c284383 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad00618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e0960e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ec28ec5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6048fac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a12d012 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f99e598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e7fc08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@325ea2a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e8066ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ea5e9d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44085fb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@304b62f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e4ca543 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fe6100c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@399505a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f866973 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbb17d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b92a79e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a3b1b49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36da248c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f730b73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7143ef54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33605366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5479ffc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23264ead +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d713cd2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3bad79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6727dad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7665a3bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3236960e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1347b554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4aef14ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11cc360f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@174c44de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@390fae76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf79f86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7bbcb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee46dbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4adc1e08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de0a37a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bd8560b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76fc5941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cfa35c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24f9a65e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d08c4c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@292ab475 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@247380ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d51096b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3684cae1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4af14958 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72bddc03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3762996e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27219ea1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675aa894 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9b2db8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5891624c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@736e3b69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41423398 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631630e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c6bdc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f17dcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62ff46fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30bcc35c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35910877 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb7242b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e74bf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4217cc16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ff8a693 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a5b433b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64cd1166 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18461695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2230f730 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32e22ad6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea85bcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f20033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33defcaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c9c49d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c43837a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a36f559 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e496f19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70faab61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52ae57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d131f39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38478225 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4387ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@876a206 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d598ee6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490b7bc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48f6f1f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ad1c79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74029294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66c522bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19c33eff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@410a76a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64e6134e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b7915a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b4df2f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a673b07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ecf3166 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d78d727 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59666411 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a4ac339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d098f17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1613a87e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cbf1041 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f047e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61800dda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6adb09cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e6221ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c4f9481 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dc0a60f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe75712 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20747171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ef0262 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@387726ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dcdb975 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eb1a444 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@766541e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77a65bd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47105246 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14504d64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4866645d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ddbee6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489c0ff3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eb140e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@250e93ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7774b1ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fbfdb27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b5d7e31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a9d1d57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8362670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2971b2b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49bdfd5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf8369e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c48a37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@660454ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40753a41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a03a817 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c701052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58171c8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31d992fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@567dd7c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a13afdf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a802176 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b318ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc80649 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38f23138 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d22cc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ceb80c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733e2f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aecbbfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@585a2045 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58370e4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a71e6ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@657d9d4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cacd1af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c89ef03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@599824b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50064d3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a2dcf3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@199fc28d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bad0a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63b73233 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57d75556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30bb6e8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67d39ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3daf1f94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50bf8092 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b9b7338 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@404c6f9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55a8bb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@294d2090 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@492f8432 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a3c1a3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bea4386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b63adcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ed8a881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@443f8ea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da003d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b4bfe5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405cdb13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78ae0f2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@553d2562 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4110a0b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67326fca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f6dc366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5347eddc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1796926c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63aef36b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62d96a8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9c5201 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dbab44c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@391b3f3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10bf87a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cdfd3ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bf10ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64bd15a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8d3041c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a11fe50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d5b79c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac994de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aaa8c7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f4c0c04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34fe2f26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a72203 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cd09f5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337388ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@763116b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dbd8546 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75afc9d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a341b71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24ce4a50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5786c3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e12d41e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7407b93e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22cde124 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@641240fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb8ca43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@353f52a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b641e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed5d394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3405fadf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7be00cb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3113e362 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@117acfa7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a320c55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@714c3419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f6ad1a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@182a3a1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0dde8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@129ccd51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bf2e17a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52c1c2c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ca4aef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2248e322 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3d3f20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72363ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@762f8e6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@139b2278 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5498cb14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17128ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d443f97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@578656a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d4f72a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@463ab550 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7abe8287 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fe8c00b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de7097b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ee4a3a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60c0ae43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3971d2c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c98a84c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37aaedce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527dffce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@652d90db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4e8efa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6acc20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d4c7ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8762445 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e742ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a86911f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dcac404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@542791f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4d41a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30bd2f1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b6c37ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@632cbf45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47489aec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ed4a8e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74ed2788 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@723d78c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a10396 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8926ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c59954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e27382b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37632fca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34dee1ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f2ad14c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@383b4ee2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16082b50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5518cb1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c21c044 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8fdb7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@798a3756 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ad97366 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b8ad99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@783cd7c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a46f1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71c7742e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c7edbea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@262d533 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@714d9ef7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be1e2f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512acfe9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45f543b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58d2d7dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c107871 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b42b79d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4459194 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a67c2fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25dbe833 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bdb4d4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733aae94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73ef23f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d5e089b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ceeab67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a86e460 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a594a05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27ce8505 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e72b708 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43bae342 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23539052 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7066df90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b97fc63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f23b6a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@604d7c16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2611b5d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52113195 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10450bc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1be36813 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3077e18a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb1e388 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5392f63c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5331afad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cc56ff0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388c7a43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c6b4878 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22f9dc9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b48ddf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eb1ab48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@283f1a6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ce26d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356591e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368d87ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66b1173b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53fc5589 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8f3c8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10e770c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e01a52e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f60e77a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c38c954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57f2e755 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24a90050 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cba7888 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56727e22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16faf3f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7abddaf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45fc831e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ba76ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@411e504f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55603087 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74a5bd46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@277bb021 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14d400a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40e4eebc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1972346b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637275dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5717f053 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30616503 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3838ce08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48948046 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e9746f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b1b540c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65ad91e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d7abaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6276ecf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@230c7f85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aa780d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e99e4d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bdd8285 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b6d2c2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d5aacb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c346ca3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b5a060a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9345b01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41437196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2efd4c69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ac75ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e6bc80d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@647ec197 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20244775 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@565165be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@361ae4d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@247ae9f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26671de1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc6f006 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a1bff8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69dd8075 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774b00ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bc19895 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee3dfc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b383640 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54d9b74d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63eb8873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3164e18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39a6e20c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@205c49aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fcbf5c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e9c1ead +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20a42ff0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62773357 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2252bf3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf5a4af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dafcd6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3699a607 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18554a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@401458e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53c32937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@281114d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31a4d0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@612bf6e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bebf163 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bfcf76c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5fa13b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@647dd066 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10e95563 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c429785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b33b74f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5147a2af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bdfb7a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6349f7e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2506bd01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49b821d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2cb5b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f6e30f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388aac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb69de1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ed103f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75276f96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33bf3d75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd9f776 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@358c723a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b99f0c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31073e51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72a60d40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24edbc8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10464733 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183b16dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ab5d496 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f50a90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3128a802 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7165c50d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@580bed3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e9a8579 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a065210 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1514abf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15e2e9a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fb9172f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d74c1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37ac2c3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@350154b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2363d58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f3c1698 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69a738b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@563c15a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65532eca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a847c2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a36a176 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9178cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@204838eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393455c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66567e81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31563596 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50b8dc37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bb919b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eced435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb4a8ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@87b867a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b0c4271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630b5059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c3ff97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c5feaa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65292e93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9923f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1be140b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6898ab99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44695daf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51fa484e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@408f4625 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d66f9ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63ec45e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60d7d9e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad305b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a13faf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e3f956 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f253048 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d99508b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14cc48d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a7b2a56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7caaec21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b60e637 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d9ca143 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee6193c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25863725 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9fb2cfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31d51239 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fc4c2c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b6fe304 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ba3a059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527e27ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b850db5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f16b602 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49616cd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a827264 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39392224 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b535a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f0c4e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28189dec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5840b2c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6262ac6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35e29327 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f99f8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@472a886 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73044d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@525e837b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@474e5375 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5669e582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d0622d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b37cf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14d3564f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d0ff4a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@94a3e24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33a4bf3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dcbf62b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f13bfd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1770130c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b1393 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6323d731 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@311c399e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e3a4c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f1d8c06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27a468b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527a4f93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698434aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a6af820 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a74b7f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af273f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@629b3362 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f9d6ce9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4059c995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@798d5f8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@423673ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e8b574 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68f3dd5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9196eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a49abed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29a9ecc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fc6c598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28972645 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f46f679 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9efe090 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606c1e30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7158a976 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29756c1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f68f2c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f75fa87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd2eb06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69c54fda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71236ef4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f20395 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d845276 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e7c6983 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5442e476 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@453dbbc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af546f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4798145f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43a398e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c716172 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c8368d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bbeadf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e9d0142 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69c08312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e82ab1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d5a04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@260ccc50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4bb90390 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3098a38f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a1f22e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a311bbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19172ee8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ce6729e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fcd45d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d335365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df35645 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22deb9e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7189bbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40775846 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27516f73 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7017fbff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33917305 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49dea859 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65586ce6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a5fabb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12eeeca4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e4e12c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8d774b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f82443b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a6c18a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64eac53c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57ab47fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ac562ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40b6c857 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72c73658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fde1b49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e3fd61c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38bd4bfc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7383df45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c88962f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c373094 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e876045 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7313fd27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b922484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e541ef2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b74c435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7762f8b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d8e54d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33336c18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@edb4327 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e9cd575 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@316b1c8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1910e516 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d065f68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45390cc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cdfff8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32798754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d7b636c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4029290f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13598d19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de4fb01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9295d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@727aaafe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d14c0ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67621abd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b13f86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fbb4934 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24a3f786 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e3b65d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@522feea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2728ff30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c0c3394 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52932f16 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe4d142 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fd570f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62b8600c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299818d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70afa262 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c9167e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65d0b6eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce76c0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d66347 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d20efe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6da76744 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a17c9cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68e94591 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62bcc0ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34f706fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50cae81a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72adea5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7610fd14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5cf5fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd14893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b3aa64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@113a133b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c54d7c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b36930a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6abe682 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47499ea7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d09a342 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22e8c32a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7baeae03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7099ab77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50aa0595 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@343f6d39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a6f030b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@710c522 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@796eef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@480f1d96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aba28c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b02c567 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b531bd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b005280 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5027af37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9371a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ebd7659 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a0bd5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a93e888 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cdbde4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59517436 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bf9681d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3b0042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c10b241 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e329331 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@efba56a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b48de7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@511fba0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5e0003 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@685d32f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345a51ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183962c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@532c27d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52676dc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34d68102 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672af217 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550a03c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cea65c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c343dcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cbeda14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9affc64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668444fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b43d7a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@157cca1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ed5c33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@330a4d9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56914ab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@691ca601 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@748d63d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@150aae6e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49929283 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4532191e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a3e59a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a07f551 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47dc7a18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12737114 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a28f995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fa02efc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b889baa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@327ab61e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fe30595 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@512a155b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e00811f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@492d0887 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb5739a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f342dda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35164f5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14de004f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21c288ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fa62bf2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18cddbc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cf57ced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4945067e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67925d1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11f791ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b2852a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27251d74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@522ac62a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f7f4d7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db60c6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22777a14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2668f3e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f976c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb1e385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278f999d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4619b949 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3eb011 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f22c398 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44baf290 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f62c8bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1487577a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1db07871 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e8d3725 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5773317b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79e18068 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@678c1cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f2b1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e693a87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b7e01e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c93d8ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@456d13bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4257a493 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9eddc2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b9b3143 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6460f40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@794a4027 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e21cfa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc2f406 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ac8dad1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a8f2d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61e43fc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a7d853b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ffcaf7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3077aeb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70ada569 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60fb21a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7edf685e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73506336 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e4b16d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15afe669 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10b7e11a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@608e69e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56455ef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cb376f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1323ef44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711de8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3f41fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1ce43e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c969d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fdfa09a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59e93a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e80839 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79734a8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b8afe4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b22975a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e2ef7c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d6adf9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19e30ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e59efd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4750bb12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c1d3264 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf4e0d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d42bdf8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f881169 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4af38aee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14a0100 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@698fc1ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f386ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4001ca67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11929166 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@687b059 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c4b88ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42d739d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@474b894b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cf5ec83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c28360f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57cab3d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5069b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f12154b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7267c0e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7031916a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f634b03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3054a9e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cf84f8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e2bbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a053e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@302ead54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9428cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f868973 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@589eba78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d1fe832 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32e2791c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79e4a322 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54848b1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3638b7c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17330284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b124ef3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3ee0e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@484be6c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e3fa6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1edf41b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@522aab5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45c02a38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da8906d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@799eb09f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e69ac84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3abcd89b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63ce0c59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fd852f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce6de04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ad03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13fdd596 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1f62f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54638dcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41dc18f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cf0343f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@793de997 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31780e98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1042b3e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d2e2c1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bc90978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a34dc57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fecbb42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13fe0850 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@608f419e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb5efbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a973c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45f643ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fef695f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c84a511 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11dee629 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66412fef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c007e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb59e8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d3cdc7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12edba63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50d01898 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6726a5f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c625a5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a3c910 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53671524 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc7bdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f626904 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7214d6b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19b123e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb1d993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@767b5a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53125aab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@432ccd18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36a4e9af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd4d026 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dd20415 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@683344ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79cb2182 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53b48b7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9bcfb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cff4c4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c98b563 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b0022ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b368e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d2d287e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6407a915 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ebaf029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1302a0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ca8c8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5cc026 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55fe5a3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@559e42fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14713216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e069d4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@198e0f08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea00751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23dc6311 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@293a76c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4025daa9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e81c773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e05e759 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44f3526e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a6d38cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aa7d8aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b89c388 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@153788b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56df51b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32d9a141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3749a111 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e12e56d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@326c3586 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dc05923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@377b28a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4b9aac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5858e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@229e0d2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c4e07ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb27a30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1029868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53270a12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fea0ee6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52b48ad1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e08937 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cfc6c43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13deca04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@654a38ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a0bc03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f142e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c1a2b30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3821fd47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc20ee4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c7a93e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25595ebd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12255494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@199bdd1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cd0194e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@339308df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@baa6bcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9e14af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10586e1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ba6593f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@630ab985 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f8df227 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf6677b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@510a87b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3360efcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34c3f4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c3e4de1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6d796d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@847f7aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a930fb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248c0647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73fda5aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44b0f6d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@362c8778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@725611c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c183ad1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f5d17bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d985268 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f82b42e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1261dfbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55c6daba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a72d978 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40fb33dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cfb8049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f0b9266 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@434e25e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a329b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a4ba147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393080ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b63f31f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45438768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4285471e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f0ea954 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be146a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3240e472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a95e893 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505bae9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@316e2e78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4383f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711c9e37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3ee1c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c630dd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79647faf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c9cbb92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67a75716 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@430ee380 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19cd6549 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10041dc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d46e738 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79e01ff3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bf5caab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b2f7449 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@412a2b33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ccbb62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dc99e42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d042c8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d67e7ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10ad52cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c7f512 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ddbd1f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ff30b7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35322345 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ba32f94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28947309 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f6b15ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5be72919 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9ea727e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed5471 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258c4753 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47410d1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1de2fe0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75447f1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32b5183b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dd3eaaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774b7990 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a15ce1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b4ffc2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@592bc757 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7500a21b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eedbe0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a078771 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d91fa19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6df193ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e5f4db4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78c56501 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a161932 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b98a879 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f8e7653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c30f3ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26b5aaf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68a0153c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34c0da95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f95e140 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fc3873 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c1ddc25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ebbbf41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ec4d237 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bde423c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19e04196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ae2a45c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ef805d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b89fd9e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30fee588 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ff1778f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e2fd44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65e385eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2276ee7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56cfb8b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@405167f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39646ec8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c7452a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7554582 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c70ec5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5678d933 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5817600e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c357db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1a7f0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3b98bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d4f9a64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225d6199 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd298c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dde1ff6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c732e93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45ebc72d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e244bc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5348d65c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@465a7d10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d931859 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61553a69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@743afa29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619dab43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@248936f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc7d673 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f0f8270 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43c4ff61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e17371e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3118e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1e1915 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b45c860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cfdb5d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@487f44bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77963a14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7215ea4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23178ae0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@117f6881 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577fb620 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ae5611b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@153aba2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7febe36b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@558f8400 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c3da13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e8fd74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14b8b039 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d3e892a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57c7cfdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24b79b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75dfdeda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce17e92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5531effd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dc679a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@466447c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7146b479 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bd2f3ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5527bc25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3119e347 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75b87c61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49cb96d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2165cd5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9811ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69752508 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c1d73d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@731d4e36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c1536da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c880335 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b9c56b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2269c30a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@670f927b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b72a404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52824205 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e1faacc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25986b3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd4a90d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b064618 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@380cda7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eab97da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4029d1c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d67958 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57d78d36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5540c551 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b69a1f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61cf1f7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a1a904d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5923b612 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd3e2fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56fe289e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8b852f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c6fe6d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cfb29a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b66487d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71839217 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42e98bae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79a2c5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eb6a970 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6348919b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d72c2a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b5ceb8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a28b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2779a568 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce52cda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b9947e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51372b30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b0f869b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ef9eedc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22f8919 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a294cc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2dd52447 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c5bb02f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@524283fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@722da207 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76de5719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7755b25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ee7554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e4cdaf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a2e38bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77affca8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d973dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dfa8ae9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56ba8e0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e7862ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5031bd4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7935d863 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bfdebae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@660e880d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b08924f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ca9b38d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@781d952d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a4124c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1524492e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b0a5dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2010fa9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4394b567 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74cee902 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45613c02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6282da1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45b64a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7de0bd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b13b601 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f35d757 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6014fa51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e3cd2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4116c26f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50883a75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17099447 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0aa127 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69781d33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b2d9fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23c2c106 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@688a5fc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41b24667 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fa122ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a5e0ca8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2010347c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73c2df67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55d00fce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f817290 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e1312c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bcfcf8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb40142 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a95e3f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c96373e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a008b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1790a4de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22af6f89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fad8325 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1651a7a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f547e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d7fb3d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea8a7f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@255f236b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b3bf226 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b7a69e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50968c13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c8ea911 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58f0d064 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20bd9fe8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@442f79f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d10964d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63bd027b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f33933b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10ecdf9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29779132 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2252f0bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11cb1587 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672d381d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72c6dbcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a40a36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e8a4868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e97474 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4435ff21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23a7b82b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f327d3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34b3ec79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f94c139 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ec4c00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ddb1a87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@620fb61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f447f95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cebb031 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29919609 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28eb9ba9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@740fe1f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7564ad7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50e04c05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2422a363 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fa9e640 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f76b196 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2da44a19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c2cdd75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cd7a356 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3667575b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@391c270c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43a1822f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad441e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e7c169f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5600741b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@197f340e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@719a1e8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fbc26c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a492989 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af6bb8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb99619 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75cd73c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76cb0998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@697ed620 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb08c48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@307df16c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e54b932 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@104b30e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bfcadaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d5cf28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f7365a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bcb238a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@259452a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f6275b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b6809d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@190da84d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e8bc8ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7228f927 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1965107a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dde9cdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d5057c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1884628e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29535724 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30723e91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fb3811a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24981070 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@174189b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36e6bb7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@355e57e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a1daa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ab6c52e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4988a244 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eae4d8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5105fa3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2de6ca85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21f0c7ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d4e1c56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2364c9cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@384d8dc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f257a09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a46917 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@370bc419 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@637c6cb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@573da7e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@464080c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f02af1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f9b778c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a3cf31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57a71bc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de73c6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19b4114c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@12e59f0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d73d7a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29a3a807 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a4e20e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8702e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d2a2e78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79fd544a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7e7de9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58f8ae85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44bd120c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577141d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a5dffe7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@774cf8af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bcdb938 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ef247d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c84180 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d158d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25e327ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5023e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b55ad6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2db413f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e3bd2c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65afb9c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a76ee9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bad1f1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6be0514a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7319845d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b217cf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7817fd49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11948900 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e10dc03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e46fbc4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79fcb4f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c95ee46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@644dc313 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ddc9c85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711beefa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@129fd941 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1840d55a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11714a63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3fb30a89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530a89ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d7e7cd1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5647054f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7107aa4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76c50072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e3628f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26e06ba1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a79e0fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e1bb56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c5505f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42dd9265 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6706dcfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e04f040 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@138e0477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d6ef2d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c5e30bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15214b86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76ef48e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cdd94d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@219b9029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e7982e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@325d2dc1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62f66c71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f91a2de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d0d7571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36657e34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6721196d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7534c005 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7314852f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e1dc1a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1984778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52ff8ba6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@515512b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505b8d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46e59c88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68698d49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@475a1566 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb1ce5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5da0ef8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21fdd8cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@361b18b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b9d334 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@241aaf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d3521d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23856647 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3abc2170 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@352522d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@517aae17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59454d21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a940d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e86c1ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a08ce2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69d7a40b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21b09669 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9912299 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b4f43b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@117e57a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24b128ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2434430a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56537e9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68517d1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@450ad6ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43a0f113 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ecaf85a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d90dd10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ebce187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ad0bf2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63018802 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@892a12f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@113bd963 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e2d0d81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4103c1de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c859b08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@133ab13a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55bde7d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a6fd181 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac2461f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c379b1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60e306c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72191af5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a835817 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aeccf20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d960141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57e270b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57a1c9bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bc889a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76891995 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@322ab020 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b1f6beb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f55a85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a0ebe3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc2a2ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d92da8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@639d80cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c2d35c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b03778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f2dd7ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3039833c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ff3bc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1011c9f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4153cfb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d8f84cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3122b1b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f62981a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7093bc41 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31fbe772 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15703f84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee9d778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43f51750 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ddba82b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39c32e0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f16c5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3255bfa5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42bfc334 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622c1e1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@306d1d8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fda24b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1340e418 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@111ed480 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49c7026d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37c66065 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23b222a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@446f4883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@354afe09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@318b658d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39efe37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d8b8597 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd38e99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359e3fe3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41051021 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c392ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6327676c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27823dcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eefa505 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720520b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ac13592 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c9b115b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fef4840 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74b4a5ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@577a634d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132fa130 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d7565d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@179d10ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13f9791b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d317c75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@128b830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ecfd0db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26264fb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50825ddd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14af96c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14e6d583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@290328 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4287ca45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af6656e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@89ee77e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40669506 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ccfbaad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@258b654f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad1d397 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eeba36e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f306ecc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11efcf78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45f6fbb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3675c0ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5f9eb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35241e8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@477d00a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f1568e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7999e79b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c5c766c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@438b2363 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@372d0993 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13e95d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e5e7c8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1454d61c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b6a3494 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e8fb82e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43c81174 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711ecb27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b4a9f04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c7048b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b87f356 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c4742f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134a5ac1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b9e77f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17b0b91d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17639cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40abe776 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ce56203 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0f4ad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8780b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31885249 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@165adb01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e752cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f78e6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@139e8222 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53dcd183 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b65b039 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@473dc1cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2055c49b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@153059db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23e2c73c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@af8512d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4073df5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@136e522f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2680ac59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e39e09d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@536c803a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f25f52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1133df31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@302fe882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@775d3ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27dd149e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28989d03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67bc0b6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eff56cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c9d26e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5537d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a35864a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e48dcc9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11dd3fcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f26999 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b3a2bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b858a90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@735ef773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35a6f620 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b0e3c55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cf8f0ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1819692d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a8c9bf8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50aae068 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b8c0753 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@588f470e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26a254bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@218c49a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ad1645d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68c4fd3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c77432e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ff5d049 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e47bc66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1125879d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772dc8a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@466b3990 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252c09b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18887be0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@179e8bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e41f6bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b653575 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e90282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e1b3e22 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f7efd74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d42a573 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65f75f4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@720a87f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a375191 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e60e99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db41889 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b277589 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7c14a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@293c5f95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34517a6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14faca55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f382d53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44331bd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0ea319 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3964664f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77f89685 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b49de07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2551a61c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f9deae7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c620f2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69be5142 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41777b2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11ed7b34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dea0551 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@164561ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c91ce0b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@165f2c76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b427361 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@763529ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@143650a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b65e2c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23e45fbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c41e0ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad42cb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e73f680 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@feb515f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cde96ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7948449 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1642a5cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43767ff3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b96e9b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68039a42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bec3642 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b15452b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6732ccbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13979b93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1943e185 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@284ed39b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59e8b50b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@760cf6a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c56e2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec9b9a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3295019c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23bd68ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7852eec4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bb544cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1362fc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4906df57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26cd3c7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@176f9f4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2717d271 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58945da3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1775bc18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c0bd445 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@659615ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f9535bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25bacf2d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b19e23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52ddda57 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d3155a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17e36d4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca8a913 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@209bc7b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c9e7f3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@501bc0db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9a21bfa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77fae322 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58321d65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a113a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fc5bf45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f06b66c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@271747dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79f56c8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b28db61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4147bbbe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cf32ded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e30b3fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@539e928e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ff84908 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aa1855f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61854492 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4628b0ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bf5010a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d18f256 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6728eb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dff073 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a14c93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ca18453 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb48462 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6492a3af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@685bae7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68582c3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52f522c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@230b7ce9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16162e8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f86c9ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@449e07a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c0ad7eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bcffca8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@128d8dd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@269a030f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c452db8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@288ba224 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@561669cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20e29a7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@452f5b33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c05b30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@557d1ced +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61786d72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e3154ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c9a8d80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51bd6e8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@730c9a32 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27cc3cc2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f824336 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2571a772 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32bb64e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74d7745d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79c10202 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51a3707f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3856a2eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@776c4c20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cc81dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca629f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@416a9b76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5472c237 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36d2730d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b463d95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e62bea0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c43ac21 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@392cfebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@437cc69a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8a0f7eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@983607a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5704aef3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7269ea9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df94496 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6feb7be3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6a23db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20ea0ffb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17803284 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@230067f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f38fcb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f5da37f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@95fa602 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c05f1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@135cbca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7381c6c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71a110c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c89b67a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cd43ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e07728d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c29e30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4899c714 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21208d2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64f6009d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f8212d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fdfec61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ab34349 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e95466a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36fcfbdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1660b586 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76a90508 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@728baba1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f89c37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30a7d14b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e980312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@216177c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@469f4efc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@483cf2ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@672e86b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c874809 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10531bcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f5738b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e77be87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38267c40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30a3a5cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36b7a9e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25a7aa31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee806b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c84b6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8bb3a6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10d5ef6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e66ee02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f344180 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8d4c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@511c6846 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18c39a99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@566e040 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c2ab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33602c88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37736f2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5ed61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@659c36f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57aa42a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b0c456a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5168618f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5547e6e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@507d3991 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@494301e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bb5755f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@312fd80b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@827de0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@593a8ae2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29550317 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66f579b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50941cc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5df4e1a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cb81cb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11cde658 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@614fcaf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4cde8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78d1cca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@790cd545 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668bb0f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b4ea20b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74ca86e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bf3c3c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50c4ea60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ef44489 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ea91815 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb10dee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c517d12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da21d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56a26bbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359fc2fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22b180b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fd1c3b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a6b8643 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a82dd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fb29155 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ebf0b59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b0c953 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7876ad38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@335a35a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550685a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b023404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66d04331 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@485103c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@299b94db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@130337f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ae2762 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37216546 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42691235 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@485f5177 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4cea29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@727bbdf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7889f133 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a6fc3bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@635b757a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c60c8c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b57b811 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f11f6e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7759bad9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db56205 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b64d292 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b82bdcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b75717 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1704b86e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a29085e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64eba00a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f5f4c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6daa5822 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f43d631 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5691c023 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7388c2aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c5cefb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ddf5a55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65970515 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@503f069f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b354a4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c7fba8f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@611dafb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49be46fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@180812b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@596268f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@550e6c0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e3c79c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36792367 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c16c09d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5ebd0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57a125e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@283c566c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e479de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dfd165a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@132d8404 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59e82f3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18839751 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de90ddd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@421bb3c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a12dd4f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eafe043 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bdd89ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34c7aaa6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7873f1d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ffab61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e28c909 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17b3c2f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70a96c83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54a97554 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c919e49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bc7969 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ecad029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d60171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2135b0d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4724e2aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f0729ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4b68e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32f4ccea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@522549ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44fbcac6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21929eee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359595aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d05c29d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57329c01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e20de7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc3fcd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18b1c81b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b208d98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29df3e2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d9e429d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bf9aa25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eab08f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5414c072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b21eac7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d9fdd04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c2064f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ec52656 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a56989b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39221528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160af118 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@610a9219 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f165ce3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b48f7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7773483 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3258f42e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f7457c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d59fa3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3334143b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46b86ff7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bbfeaa5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@497571ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a733136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f312a58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2baf19d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4937ff4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a0e9ddb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c3b1528 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f89aea7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7932ddb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac670b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bab083 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4184dac5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e345803 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a44e6f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cac7992 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b19ca30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77e17110 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1aff6273 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6560e857 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509df681 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c886473 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d85776b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f451cdc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10d3ce4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f178bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19cf8578 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e6bd961 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10c4ed71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ceddcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@364f7688 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76af01f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4137c530 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64325067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2984915f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@272abd17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58cbe287 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e8c2091 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@238f4276 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513f8115 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56678e6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a0b49a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4065c92a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dffcf1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@701203c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b995022 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@755be6e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664f604c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68c1ac79 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3908e903 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@134e7436 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e1e3b13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b59fccd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15675054 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75e8690a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ad11207 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e4a3804 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1002869 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3716a17a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35d5ccb5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a204813 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ae3725e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@487c1709 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d7dad7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b7f706e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a5f80dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62b312c0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fdf7d23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6fc380b6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@592cacfb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f3900c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5731d734 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24042b54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1873005b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@273f90a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e6e7b0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bb5a11f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc417ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20256ae3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d9edbf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b752a0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20d509cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b2c0d44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70364acb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22413b8b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a52c57d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61dfab5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23fdc5bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8cb2f2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@340e7a62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2897b2e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@650a526d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e98b90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73f32a05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66e6a61f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51c80994 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2985ad36 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43cc0901 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10499939 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15ca60d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4570d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2693d2bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7acf1701 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e47ba9b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ec9a61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d291ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a7f9c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20970844 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7df5b1a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@683d0a0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5468df70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ca52891 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c8dc039 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c60dcd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee10376 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b65539b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@199bcac6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3872b54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fb62e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44c3004e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3493fea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39ae695c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@671ce8e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b81553f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@717d8bca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bd3e830 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b084c50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a11dea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17af37f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3399cf5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a0a7fb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eea5496 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2094fa0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d0a3a64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44e42998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58b68b76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f755ef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53a08666 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@366ddd63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a6c1de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f9b7ab6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78df5a95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20810628 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a3884de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8203e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c8be752 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44130cad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ad37e85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64b0cb11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57b211a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b98099d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a3e793d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68b4c328 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f4ce09d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6453d008 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dddaa2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@533e973e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b3ecdcc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61979d33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@116bd7f5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36cb2581 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bf6ce39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d5ae71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49ce1521 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26c6d6b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2c4353 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60a1da61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59ffbc49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@491ca2b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6083461f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e1ffabc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a2612c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6831c30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cca0141 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@df1de7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175b7d00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9a79147 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@254b593b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@264a71c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ab5bb76 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fad32ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20503a81 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55e19b28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1248f4e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24459f2c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5603d17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c01de48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93b869d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10fd469 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@583d9da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5722b4bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@278dfab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dabef11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66fda477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24387b40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60797deb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4288d592 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49dd0fa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ed89fa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56539bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c548c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c4a719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8b4976 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668f294b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d27de24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@485e44f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@580e6472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47d4db49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3471fb69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e3270e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2534a8a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44db31d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fca3098 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e2d2dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62133fa5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f5b9fc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@513b2e44 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25bb628d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5eae8f88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da8863d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7569dec2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb875e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214069d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18609aa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23c4d8a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b1bee06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49374414 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@551e7a13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509aa1d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64451e18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a1f160 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3149df65 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345874a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@782109eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6013dd7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fde94ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68542ef7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@184a1c47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664cca1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dde4bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4434ff04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cd46be1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c484444 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73377ea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10987315 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f3c8984 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71189043 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@399272f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ac1be0a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31ea7a7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4457e89 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e6aa935 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49bc631c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b67d1d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4794305 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fa161be +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55af59d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40d3dbd7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e7cc9a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a24f543 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@556a7a23 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b390e3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63e0ff2f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb390fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@388e5e6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b85cf9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ff85e47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dee113d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77d965e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2275a5b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fa2cdd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32884ce8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ec2e918 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@115ad6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68307c9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@182f927f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bc224b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2deeb1eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46d1e45f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22893a90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4247faf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1afe6aba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53c486a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38727bf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@451f9670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c5d9fd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b5bdd48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@705db00c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49c80461 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9aba98c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c1f946c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@115f8833 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18bd6d75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c8e4027 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c42112e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51b10885 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22941103 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ca5f20e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30c25bd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69ee9868 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bf0805f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58f63aad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a48de95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6778346c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@865a2a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61afd257 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5daf0481 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b52800c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e077d40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41a521bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@640bbe04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bb61069 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e3e8122 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67eed91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f3c98c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b2f212 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e1f4437 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f7c3000 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@345d7f93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@276c2968 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@43b6ff4e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4dc43b08 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123a64c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54db3882 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2063f1d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c9382a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dced3b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ce5a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f0e2944 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5812b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1470279d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73669160 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d6cc241 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@128eca86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d75b85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40fd378e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b588b1a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f66b80d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4250cf17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bb8c194 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e787a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e0be7e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ae7a742 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@177e75eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f2c3a31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3eb9e2c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39638fea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d567400 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23552029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@520b2929 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7732f05d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19aa6ccf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@137443bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18bd629d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@786dd9cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@326b1e19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3ea74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7069d915 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@665cc576 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cad8472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10512fd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e4efdb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb9ea7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e5775cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57e1b438 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9b4836 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f90a19a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a9c50b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@400667ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14085d60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7efeb20a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61f7cbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d740c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2362f041 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@288778c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55000bf5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a1eb0e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ecaab19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d18bf55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@689254a0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e468f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c0d897f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49763bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e5a10da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a5d1c48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a8b300d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@753e3063 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b4c22de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69c1daa3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ad994ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee01ede +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4400990e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23f357ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15bac8bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bd2323c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bda1ea1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@122f681a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@511385f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2baa0e38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50fe32fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f8ef47 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7489f1a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f29eace +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ee16033 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@509d207f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ba2977c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b9d7f5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ed862e4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37ff3598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48995d6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@83eacbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45bc301d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b218ec7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9d1c0f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74f7c638 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ef3c63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57bfc70f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@506e5b5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae28169 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3347ba72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae984aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38ffb826 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36988f99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38023712 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e98286c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2be39311 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59f535b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70663338 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f12be10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@685ed5f8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18762ded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59482228 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1360e52f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d0f03ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24bf4ea7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c7e717a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2823b247 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dc4c69e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d071f7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c6a3556 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@796e1716 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e5de93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ba48f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fe70d75 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16386f38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a052a2a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a28005b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d0324ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1817d3f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd0eeab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21d01d52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@752168f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@538fe5b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fa6ecd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e3d50f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a5d13e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cab36db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4403cfba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50ada501 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eafb20b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b475eeb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e33e26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cdb0cb6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f67d4cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ee67fcd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@163c7839 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@223678f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@732619ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ed31874 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bdb4627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a3b6270 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3990ccb4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f86634a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eddf0e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f542baa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@101eb798 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c8e0ae2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7db5ab78 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7332c61f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16f51d96 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ac0dcb0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a7e46fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f996f95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@713ed25d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@de7b596 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55f6634f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51974ba3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@563cc7f6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@664793a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d783e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6bece1c7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bdf983b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dfca61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a38deec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66d6f8b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@416fba14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e9cfc9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@127f8a1b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@433dcbb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fbe66d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@261f3d99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48de5413 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@65137efa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69174c26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36bb4768 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c7ff360 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72e5a8bb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2140bcbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d2c3ffa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ebb31cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61c4f6d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45450b01 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@693a3fbc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e684606 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@485c536d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@252c5464 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1dd358a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51476568 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7baa9316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ce26773 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bf32e05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51f57036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ee87b3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@519e426c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733fcd5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93177cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34bdd445 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17936d9c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a4b171f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18469484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42037435 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e700e64 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c410b5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@208f5ce2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7974261a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3739047e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3326c218 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6914e0d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31885f1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e6da7a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b5e9a70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582f1b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76012eaa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5572ef52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a407296 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78dd595e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d65c91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@451c2b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c2125b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ea49ec2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b236f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f41336a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a68979e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a04a9f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e9f62f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26155f3b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@688ff800 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ae9a33c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@357e7ac0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@399e07d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e46a718 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@198111af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f6fc025 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1686f484 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@de3ef2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cfe8b51 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3db10e03 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1cb2ffe5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e3ad1ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6676777a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5302bca1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@58c42cf3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a92bad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55314165 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@139018d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aca0de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2377a754 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c92e036 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f5711da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5548bcc6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b1bfc3c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20633639 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ed3a2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67530699 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@733dfc53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a56def3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@54f4b1dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bec1214 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37afac5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b725d1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af967f4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71ceae0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3bc93136 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@620a11fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@133d0c8e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17877622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6535b565 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5702ddce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ca0f2dc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bab70c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@486ca4e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9c1fc8d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@225cc97b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@770b18eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5aede155 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c4e4b11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189d46f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@377ed94a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22c61b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c14b42e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ecccd88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f39fb92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e797065 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77450e40 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fdd3dad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fe8aaaf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aa70b6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@754f2775 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53594c24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9be2932 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1bd0a719 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17a84f99 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e090924 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7606466e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68f0dfd8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3df191d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf7ff6b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f06f34d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5166f56a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60a8c1cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24606616 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c74754c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a5f180d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cdf1e30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b28856b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44794757 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d4b0a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62ee1ed1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6de2dc3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63af7d2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30ada866 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22273034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59df3b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41016b5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c321580 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78f30a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27aa2e1e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6dbbfa2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61296c0c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fdd5aab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f15d90 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ba9a00a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@714afe7f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b7faff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c92a24 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71f66f27 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5769de60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c32b670 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@420429d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49e9e8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19caad5f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60b21920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31ba607 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c520925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bef5a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b5978b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e402c04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@728703c8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@563c3e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59b61508 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76393236 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ceaf67a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@270d5ec4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4efa1178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@716d2213 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1adf4362 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5769c583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49066fc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51e3a14e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31c72c43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@289bb225 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@529998a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ebcaa6a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@161128ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63eeaab7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d2c52c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b858f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19f06794 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f5ddd0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10780452 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5443a97c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e95a8a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f41b1df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fcb7af1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@368a243d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b972db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7754fefa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53ec1e13 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34a0639f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a11e07a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a6278ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49cded35 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e46d308 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b71ee8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32bd4810 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14bfbf37 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ca9f03b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8f8509a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@505d745d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39f22f66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a49284b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@607fc118 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bbbbed6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@739a33d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10724690 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e1b2287 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@850709c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2af696bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28b213b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fea033e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@743548cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ae87b8a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@527ff2d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7da3f85b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@230eb410 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a93bfd0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27d76d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45d438e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13acba7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@456eae4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18f6236a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c811bac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a883108 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39baeb77 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@107e6259 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64d65781 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f1d2a62 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41928da5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@229d5364 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@742172e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32290835 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67835470 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2fcd2d97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5004a4bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a166628 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@574b97ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11aca925 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64f8c6e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d46cb70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76b8c6f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@229bc82 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2044ade8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b92ffdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b2d2316 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@142f595d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11019ddd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d11d401 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f967f26 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@311d3038 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cefbcb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fbdc4f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23beed6d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b7a85d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2aa1a42a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b9a77d1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9b264a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70607c7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@574706ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@fbbc76a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@91419a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ae41771 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15790ce7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@442809b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d58c980 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@375d14c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a1e28fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e7ff1aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4540312c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ae4ee7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a6641c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10452723 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5448ad56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@696236a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b562910 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62936166 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73a0427f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b8f9db9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@617b2b71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189afe69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69d8e4bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ecdde68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26d55e63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f38566 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47a069fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37eed1f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b44a721 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e69cd9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a428a1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4be2f3b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55813847 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55043a70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b0840af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17ab6d20 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b343c92 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f97583e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36936a5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a9dc3c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a7a0a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15745c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38183d6c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c8f5a48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1001598 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36a28497 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4fa82c72 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19a372d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@180e18ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3aaab106 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34efd19d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3658380f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e7b9d10 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@31b63558 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b9f79ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c70bef6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e612175 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e7dc914 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e2cb038 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a23805 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cfdb0d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@601a8769 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bdc4622 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da8254c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63c19247 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@57d46099 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e50c117 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@620e11d4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@db2e478 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@79168dff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4176b3b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53e49af8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@496f35d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45b207f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70c4f06c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70087d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3771b31b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a742552 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de4efe0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ed9e1bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40a6416 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f1f8f70 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@559ef29c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24bb259b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73c6cfce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4011bac9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62b9aa45 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f53dc74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a2ec8ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1724f691 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38e08db2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@395156a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@151498d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2534649e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1905f097 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ad26177 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b31a6fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da73192 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b90fbf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1270ee25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@460acc3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@206f367c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0c5ebb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@191888d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@189f298d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@419a2034 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f8d473a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@728e5c52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13bcc0cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@337c90c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46c6fac7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@46aa4281 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@33655823 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e2798ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fc65b2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6beca018 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72d15906 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40837f7e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@dd17b46 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3e684 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b52969d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20f17e06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26fc3a15 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3ae77f88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@556ef037 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7ec4a5d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@392f148d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38b5271f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1af88776 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41494bce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a416 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@111ff6d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f8210b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@441e25b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59201a30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37e7fe84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@683f4f7c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5db9cc67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@411ac7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c0a61a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e5c8bf0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d15a6a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4eeb90c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5bb6cf42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214fcccf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14f1ea8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7018f154 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d39fcca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42cf6216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b8c9eb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c295acd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e24282e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67423e2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38641d29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eddd7da +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@147e577 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7782083f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f17ceb3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7fe4b78c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4136895a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2612a2b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77219294 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d23132b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@582a338c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b9e0500 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10509fd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25df7cbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c608ddb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37cd2b52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2377a449 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d130e1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55b54a30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bfc6c6f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15b7c4b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@70b15bf1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@faa6b1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675de193 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4ce413a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f0a71f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a5ed05f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6287f5fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e928b14 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3de90d86 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a9fe4c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c5cc782 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@668c2936 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@137c9d5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dc95ae1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6733f20d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@60f1a244 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1790457b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20391b3a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@586226aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356c5c60 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c27c2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ca53c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1546b07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568cb811 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@659cb5cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f52aefd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dfcee43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e8399dd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e316b0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47e1572 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e4a7d48 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4c75462d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@297935bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dd86067 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3015f6a9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56b79aa8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3683990c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28c61ce3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37f917ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5df1a961 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7790102a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f293b3f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10901741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40fa0e93 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@371c47e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@114a72f9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f0558e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@116e4db5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44981b09 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68764d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5581da2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6638eb59 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@627ce4c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fe0f5fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a1056ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b530495 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e0c8dff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@669bd0c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e264785 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d729378 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb8e5c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f6e7b98 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7338099f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3a18bdd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@617a7da2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d8478ae +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e43f870 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@61ca384e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53efdeb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5da09d63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ca16edf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6c19354d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@412e3c68 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183cfaab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4f3cb8e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a2a4f12 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ad7f343 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25c1b310 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f10ccdd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@377a50f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52d2e0b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a2380cc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@735ccd71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16c4ee28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53d6630d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3515ac5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2037c8cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1133c74d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@711d006d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69721d61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@771dd122 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@789b6923 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@619c1035 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@187c9072 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f1c69e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@228065f2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@231cd42f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f1f2f3e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5055c775 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@715b9778 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@342398d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@126327b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cf324e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@413e8e52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@69f36bbd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62a6a7b9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@17075332 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@227e139e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50a22aef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7eec42d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2add9921 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@786d8d07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@395935b7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48e7511d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@eb5ddb1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1a246cd5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a2006ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@183ddd4b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38c5d3a8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13e42a1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b410973 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d6056d5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bae8e42 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7876eedc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b1f6a30 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4565178 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bf5132e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34ba1282 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5494b0cf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62fff6b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3140e5a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@558acf0e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24690fb8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@393e17d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f65a883 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c14e4de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@606c2e2b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29251d5e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@b268949 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@203faee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a3c1bd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@206df73a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1f3e4ebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42ab8329 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dba0ce8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@32a1e042 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f871b1f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@412b690d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cdbf08c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5145aea5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@367d1871 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c047cb2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fe8b62e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a413f0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dc42165 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@322a2b83 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@68a9b22e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b510758 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3782c45e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5497871e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@710a7760 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@52c2c5de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@73435bec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b048ded +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3917589a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dbf8653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@144a2e33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2b9ea2b2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28ceecd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c27a87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cbce23a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3269462d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6d9e8cc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5732c5b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c94505f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24342dce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3dd5753b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c0b307d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e3810b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bc96c4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@11d657a5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347b153a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a0e3167 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49678854 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a2f95a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6918f18 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d4f03a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@281f0571 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e8b0755 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34e8a6d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@119fc1c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@417b6a06 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27ac7cd3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@568398cb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@397d9a5b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23ccfc28 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67c25666 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d4ddf95 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78b72725 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3cdea9a7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49daa93c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@aa876e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b67f69a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2bdfdc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e3227e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5764659f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23c39acf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7d308a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41ff1390 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35c3cead +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ae93f38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d37a78f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@14ed522a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f7050bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c23a895 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6cb86341 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@123a10fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ff247b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e7435de +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6179f53a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2386b40b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e748021 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71418cf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fbf77c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@490fb6d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7340525b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e1b46eb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2abe3984 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1eb3ab2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f90e029 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3be8df2e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6041b3ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63a89e33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@bd1bbc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76bd043c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b8f16e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c715aa2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3af8d168 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@24fabeef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4721a4b0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@713352bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@d3cbe63 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cc074ed +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@360e4be1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d43592e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@35642b43 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4afc561e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@ee95de3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@49a4ef39 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d8eeb7d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75ea0a74 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3128476f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f9bab54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@756466d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@50f86bab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@23185de4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21db93d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@775f6b94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64e9e3e7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ff75e3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@784763d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7448b480 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f158735 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21bbb159 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62212762 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@537c7312 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20a36a67 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5cee2653 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@379f009b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74b0199 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c80a345 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1171c638 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e258755 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13d12c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@389e1328 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27414205 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@55ac7697 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6eddf9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74219e0f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@595dbbe4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ff37696 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b8ef84 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5d91ccda +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6927de17 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d945263 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f324a9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c16beba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b960a49 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44ae4812 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29a5bc87 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@30df3467 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bd0a477 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d089dc7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@494d5165 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@782827 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e75c743 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@779a9bef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@547f3a00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@158709b8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45989b94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66fe352b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b7cacce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@669ba6d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@675ed5ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@709ef301 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e2a3b00 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@653cab33 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74dee18d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@735f12ec +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c22c5bc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cc79c38 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f96d8a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74556dc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e3049db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5ed63079 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@174c80ff +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48a894f3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@544a62ac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b70c5a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4848f14d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44cbfbbb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4393be8c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e731ebf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71be4aa5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6a33710a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@340ad51e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@526aa3c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5102204c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20c8a047 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42883365 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@45fbdc61 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53b94d2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6014d6aa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6304d2d6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78871203 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@214394c5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6724a7c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@22f35b91 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@622442c6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f7c07fb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1d24b97a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25b3469c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6541129e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5abd8212 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@626a1f56 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5dd22221 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@306a0ee7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6f1a7ea +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@34d388cd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@785c675c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18a14386 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1364698f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b590ece +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@20a4e87e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@328b329f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2130d50 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71cf4ca2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76afbdfd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ddb767d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b8911b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c197712 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d27339 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41fc8520 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@28ff44d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4569b2e9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51b64ac6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@74c60055 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@996bf1c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@160ba3d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@62e320ad +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2663400c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab17583 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e0485ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6599389f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59c66700 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@48ca0cd6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72596920 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5e28fa34 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@343c6896 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2c086e7b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@15a112b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@c41a369 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b6e12af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@347ac40f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29d36cb9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2808d801 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3585cc5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6160ba1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5364a42c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d42a691 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c3d6b88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@18e99640 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e5a04b5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@460451c1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@517213a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fdd45a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b3c17fc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5fd2c303 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1efc2458 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75c86cd9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e90066b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@67931261 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1832342f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e2a2d54 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27d5a4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7f58dfdb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76c21f53 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@9fdcc52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@37c9a23c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e560519 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6888bb07 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f706518 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75d2ee88 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b1243e6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@66ef9900 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6abb7860 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@87f6d11 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@51ab028b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b16b627 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7a5faf5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@93b42e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8db614 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e548fac +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7976f27c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2a0993e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@47d9ada1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1daa6319 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@530197c2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@357188c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3f4f8f69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@71b39132 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e53a741 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@175d05d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@542af49d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39fdc52b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@a2714fa +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75aba3a3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76262f97 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@21a578e5 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@131728ef +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3b093795 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7c94fdf4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@767cd0d3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c06dea3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2ec1ff29 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@75e1dc58 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@658b607f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f1249ab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e0bd7d7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5573dd52 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6b45d71b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@760794c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64240b1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4d908466 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@8286fc0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@42c303d0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@561094f1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1fea99e1 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5477a3b3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@e70e76e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c154d4d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@177be2bf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7e78a1d8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@77ba62ee +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1b33671 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@499542e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72d85c69 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@56d1edce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e4949db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2d2890a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1da3aa7a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@44458b9d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4046155f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29023075 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29f2af55 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4183c095 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@311fd917 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@926d6d9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@359258af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6acab1e0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@25521acf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1afd80a2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3d0de472 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f71782 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3746e171 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@41c89fab +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@13b4dd94 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29c58367 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2908b247 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cb8450a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19d57574 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c8e0392 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@707810ce +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1e869bde +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2cca33af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@40343737 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@707aa19f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5a088c25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@27228b80 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@464e9ddc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@631dbad3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7dea7b31 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4e279187 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7cb545e2 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@72f0a385 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5c6d7d19 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eb474ba +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@78f8bedb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f63c6ca +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@38d1e78a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@354d458e +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@87e2b9a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@19bae9c9 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e60a0c3 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@715ee1b4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@88526fe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@463f7286 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6df2a43a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6ba7ab71 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4a34d22d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@772222af +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@f898b05 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@59a355a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@356a5105 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@533b2a5d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@10a9429c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@53643b4a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3461aa5a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3e021749 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@64651fb7 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@16a9c61a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2f238def +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1357f1d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4de1402b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1520c69b +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@26df7d25 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cb83374 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@4b1fcad0 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5818c936 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@573a145 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@489d110c +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7d610b85 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63ae2c66 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7b4fc0df +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b7aabcb +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2e32be02 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@3c6d6abe +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@39210cd4 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@378bccc8 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@cf2a695 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b1d152f +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6aebf534 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@36f892d +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1ab051db +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@29b79bcf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@1c408998 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@63995ddc +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@7bbfe6fd +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@6e4363a6 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@76d73bbf +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5f09ff04 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@5b056216 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@2eae6483 +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: +Snowpipe Streaming Doesnt Have Pipe Usage +[Ljava.lang.StackTraceElement;@616bb82a +17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141717,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error +17-08-2023 17:42:19 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe","start_time":1692319339194,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start +17-08-2023 17:42:19 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe_streaming","start_time":1692319339228,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start +17-08-2023 17:42:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:42:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:42:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:42:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 +17-08-2023 17:42:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8324131930661886881 +17-08-2023 17:42:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:42:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 +17-08-2023 17:42:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:40 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:42:40 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 17:42:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:42:40 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:42:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:42:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:40 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0, existing:[] +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 - service started +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8324131930661886881, topicPartition: test-0 +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8324131930661886881. +17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 retrieved 0 file names +17-08-2023 17:42:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881. +17-08-2023 17:42:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 doesn't exist +17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 +17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 retrieved 0 file names +17-08-2023 17:42:41 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0: cleaner started +17-08-2023 17:42:41 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319360821} of type:kafka_pipe_start +17-08-2023 17:42:41 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz +17-08-2023 17:42:41 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:42:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881, filePath:TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz +17-08-2023 17:42:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 +17-08-2023 17:42:42 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881, filePath:TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz +17-08-2023 17:42:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz +17-08-2023 17:42:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:42:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:42:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:42:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:43:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:43:41 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0/insertReport?requestId=4b0f72c0-fc0d-4467-a85d-e02fc96a0c30 +17-08-2023 17:43:42 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:43:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573689508, X-Snowflake-Connection-Request: 324, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.340, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692317537861, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@3e569d78} +17-08-2023 17:43:42 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 +17-08-2023 17:43:42 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319360813,"end_time":1692319422485,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:43:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:43:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:43:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:43:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:44:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:44:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:44:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:44:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:44:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:44:42 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0/insertReport?requestId=d9a1de62-af0a-4ba6-8ae1-527a09dc1562&beginMark=1_-1 +17-08-2023 17:44:42 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:44:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574053822, X-Snowflake-Connection-Request: 728, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.198, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692317537861, X-Snowflake-Upstream: 10.181.19.142:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@55bc6db8} +17-08-2023 17:44:42 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 +17-08-2023 17:44:42 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319422485,"end_time":1692319482892,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:44:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:44:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:44:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:45:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:45:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:45:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:45:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:45:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:45:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:45:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:45:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 +17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7418094664318925232 +17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 +17-08-2023 17:45:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:45:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:45:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 +17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:55 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:45:55 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 17:45:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:45:55 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:45:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:45:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:56 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0, existing:[] +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 - service started +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7418094664318925232, topicPartition: test-0 +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7418094664318925232. +17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 retrieved 0 file names +17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232. +17-08-2023 17:45:56 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 doesn't exist +17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 +17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 retrieved 0 file names +17-08-2023 17:45:56 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0: cleaner started +17-08-2023 17:45:56 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319556026} of type:kafka_pipe_start +17-08-2023 17:45:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz +17-08-2023 17:45:56 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:45:57 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232, filePath:TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz +17-08-2023 17:45:57 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 +17-08-2023 17:45:58 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232, filePath:TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz +17-08-2023 17:45:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz +17-08-2023 17:45:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:46:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:46:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:46:56 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=2800bc44-1a23-483c-8f9a-034e04cc5727 +17-08-2023 17:46:57 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:46:57 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575220234, X-Snowflake-Connection-Request: 103, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.332, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.213:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@4b6bf39d} +17-08-2023 17:46:57 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 +17-08-2023 17:46:57 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319556018,"end_time":1692319617651,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:46:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:47:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:47:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:47:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:47:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:47:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:47:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:47:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:47:57 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=6891ecd1-f9fd-44e3-a6bf-b7f01ccb199c&beginMark=1_-1 +17-08-2023 17:47:58 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:47:58 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573760515, X-Snowflake-Connection-Request: 63, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.198, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.213:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@4e48ba7a} +17-08-2023 17:47:58 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 +17-08-2023 17:47:58 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319617651,"end_time":1692319678380,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:47:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:48:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:48:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:48:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:48:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:48:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:48:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:48:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:48:58 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=40112fc0-52fc-4c12-99a2-affbdaeeecd2&beginMark=1_-1 +17-08-2023 17:48:59 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:48:59 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574402908, X-Snowflake-Connection-Request: 214, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.167, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@640eb580} +17-08-2023 17:48:59 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 +17-08-2023 17:48:59 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319678380,"end_time":1692319739360,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:49:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:49:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:49:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:49:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:49:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:49:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:50:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:50:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 +17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_179094934082246481 +17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 +17-08-2023 17:50:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:50:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:50:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 +17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:35 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:50:35 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 17:50:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:50:35 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:50:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:50:35 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:35 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, existing:[] +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 - service started +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_179094934082246481, topicPartition: test-0 +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_179094934082246481. +17-08-2023 17:50:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 retrieved 0 file names +17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481. +17-08-2023 17:50:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 doesn't exist +17-08-2023 17:50:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:50:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 retrieved 0 file names +17-08-2023 17:50:36 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: cleaner started +17-08-2023 17:50:36 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_179094934082246481","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319835165} of type:kafka_pipe_start +17-08-2023 17:50:36 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz +17-08-2023 17:50:36 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:50:36 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481, filePath:TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz +17-08-2023 17:50:36 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 +17-08-2023 17:50:36 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481, filePath:TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz +17-08-2023 17:50:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz +17-08-2023 17:50:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz] +17-08-2023 17:50:36 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz] +17-08-2023 17:50:36 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0/insertFiles?requestId=130ef0ad-56aa-4625-bbe8-abb98094941d&showSkippedFiles=false +17-08-2023 17:50:37 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:50:37 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574054195, X-Snowflake-Connection-Request: 77, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.752, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.19.142:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:50:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:50:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:50:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:50:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:50:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: cleaner terminated +17-08-2023 17:51:08 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:51:08 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:51:08 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:51:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:08 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_179094934082246481","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1734,"average_commit_lag_file_count":1,"start_time":1692319835156,"end_time":1692319868680,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:51:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: service closed +17-08-2023 17:51:08 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 +17-08-2023 17:51:08 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 is:0, names:[] +17-08-2023 17:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 dropped +17-08-2023 17:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 dropped +17-08-2023 17:51:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 +17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5989379599178019927 +17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 +17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:09 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:51:09 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:51:09 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:51:09 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:51:09 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:09 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, existing:[] +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 - service started +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5989379599178019927, topicPartition: test-0 +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:51:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 17:51:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5989379599178019927. +17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 retrieved 0 file names +17-08-2023 17:51:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927. +17-08-2023 17:51:10 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 doesn't exist +17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 retrieved 0 file names +17-08-2023 17:51:10 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: cleaner started +17-08-2023 17:51:10 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5989379599178019927","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319869819} of type:kafka_pipe_start +17-08-2023 17:51:10 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz +17-08-2023 17:51:10 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 17:51:10 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927, filePath:TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz +17-08-2023 17:51:11 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 +17-08-2023 17:51:11 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927, filePath:TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz +17-08-2023 17:51:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz +17-08-2023 17:51:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz] +17-08-2023 17:51:11 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz] +17-08-2023 17:51:11 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0/insertFiles?requestId=949760a0-5805-46b0-95a3-8494d05e042a&showSkippedFiles=false +17-08-2023 17:51:12 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:51:12 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574499316, X-Snowflake-Connection-Request: 124, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.621, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: cleaner terminated +17-08-2023 17:51:43 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:51:43 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:51:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:51:43 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5989379599178019927","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1405,"average_commit_lag_file_count":1,"start_time":1692319869816,"end_time":1692319903333,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:51:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: service closed +17-08-2023 17:51:43 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 +17-08-2023 17:51:43 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 is:0, names:[] +17-08-2023 17:51:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 dropped +17-08-2023 17:51:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 dropped +17-08-2023 17:51:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 +17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8129691469536046428 +17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 +17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:44 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:51:44 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:51:44 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:51:44 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:51:44 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:44 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, existing:[] +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 - service started +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8129691469536046428, topicPartition: test-0 +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8129691469536046428. +17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 retrieved 0 file names +17-08-2023 17:51:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428. +17-08-2023 17:51:45 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 doesn't exist +17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 retrieved 0 file names +17-08-2023 17:51:45 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8129691469536046428","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319904503} of type:kafka_pipe_start +17-08-2023 17:51:45 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: cleaner started +17-08-2023 17:51:45 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz +17-08-2023 17:51:45 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:51:45 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428, filePath:TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz +17-08-2023 17:51:45 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 +17-08-2023 17:51:46 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428, filePath:TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz +17-08-2023 17:51:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz +17-08-2023 17:51:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz] +17-08-2023 17:51:46 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz] +17-08-2023 17:51:46 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0/insertFiles?requestId=5576c06e-fdf0-4832-8c7f-1a95bdcb4103&showSkippedFiles=false +17-08-2023 17:51:46 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:51:46 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573914218, X-Snowflake-Connection-Request: 209, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.473, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.21.27:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:51:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:51:49 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:54 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:51:59 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:52:04 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:52:09 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:52:14 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:52:17 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: cleaner terminated +17-08-2023 17:52:17 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:52:17 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:52:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:52:17 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:52:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8129691469536046428","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1295,"average_commit_lag_file_count":1,"start_time":1692319904502,"end_time":1692319937851,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:52:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: service closed +17-08-2023 17:52:17 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 +17-08-2023 17:52:17 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 is:0, names:[] +17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 dropped +17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 dropped +17-08-2023 17:52:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:52:18 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:18 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:52:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:19 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:52:19 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:52:19 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:52:19 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:52:19 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:19 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0, existing:[] +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 - service started +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6867286562641800447, topicPartition: test-0 +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:52:19 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: unknown bytes +17-08-2023 17:52:19 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 17:52:19 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6867286562641800447. +17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 retrieved 0 file names +17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447. +17-08-2023 17:52:19 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 doesn't exist +17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 +17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 retrieved 0 file names +17-08-2023 17:52:20 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0: cleaner started +17-08-2023 17:52:20 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319939224} of type:kafka_pipe_start +17-08-2023 17:52:20 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_key_1692319940175.gz +17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_key_1692319940175.gz to table stage: kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:20 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_value_1692319940790.gz +17-08-2023 17:52:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_value_1692319940790.gz to table stage: kafka_connector_test_table_6867286562641800447 +17-08-2023 17:52:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:49 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:52:54 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:52:59 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:20 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319939221,"end_time":1692320000180,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:53:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:53:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:49 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:52 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:53:54 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:53:59 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:54:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:54:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:54:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:54:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:54:20 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320000180,"end_time":1692320060187,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:57:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:57:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:57:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:57:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 +17-08-2023 17:57:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_28885932869605191 +17-08-2023 17:57:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:57:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 +17-08-2023 17:57:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:31 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:57:31 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 17:57:31 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:57:31 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:57:31 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:57:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:31 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, existing:[] +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 - service started +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_28885932869605191, topicPartition: test-0 +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_28885932869605191. +17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 retrieved 0 file names +17-08-2023 17:57:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191. +17-08-2023 17:57:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 doesn't exist +17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 retrieved 0 file names +17-08-2023 17:57:32 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: cleaner started +17-08-2023 17:57:32 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_28885932869605191","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320251712} of type:kafka_pipe_start +17-08-2023 17:57:32 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz +17-08-2023 17:57:32 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:57:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191, filePath:TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz +17-08-2023 17:57:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 +17-08-2023 17:57:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191, filePath:TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz +17-08-2023 17:57:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz +17-08-2023 17:57:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz] +17-08-2023 17:57:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz] +17-08-2023 17:57:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0/insertFiles?requestId=e55c4bcd-ed7a-45b3-91da-fa7bfb413926&showSkippedFiles=false +17-08-2023 17:57:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:57:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574630358, X-Snowflake-Connection-Request: 344, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.88, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.594, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.19.183:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:57:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:57:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:57:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:57:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:57:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:57:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:06 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:58:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: cleaner terminated +17-08-2023 17:58:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:58:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:58:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_28885932869605191","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2151,"average_commit_lag_file_count":1,"start_time":1692320251699,"end_time":1692320286060,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:58:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: service closed +17-08-2023 17:58:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 +17-08-2023 17:58:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 is:0, names:[] +17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 dropped +17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 dropped +17-08-2023 17:58:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 +17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4067068134490256806 +17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 +17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:07 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:58:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:58:07 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:58:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:58:07 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:07 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, existing:[] +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 - service started +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4067068134490256806, topicPartition: test-0 +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:58:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 17:58:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4067068134490256806. +17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 retrieved 0 file names +17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806. +17-08-2023 17:58:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 doesn't exist +17-08-2023 17:58:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 retrieved 0 file names +17-08-2023 17:58:08 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320287478} of type:kafka_pipe_start +17-08-2023 17:58:08 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz +17-08-2023 17:58:08 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 17:58:08 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: cleaner started +17-08-2023 17:58:08 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","processed-offset":1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":178,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320287476,"end_time":1692320288397,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 17:58:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806, filePath:TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz +17-08-2023 17:58:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 +17-08-2023 17:58:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806, filePath:TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz +17-08-2023 17:58:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz +17-08-2023 17:58:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz] +17-08-2023 17:58:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz] +17-08-2023 17:58:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0/insertFiles?requestId=773417e7-d42f-46f2-b9e8-b9977c7a23c3&showSkippedFiles=false +17-08-2023 17:58:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:58:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573955694, X-Snowflake-Connection-Request: 128, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.387, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.20.237:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:58:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:12 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:17 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:22 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:27 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:32 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:37 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: cleaner terminated +17-08-2023 17:58:40 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:58:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:58:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:58:40 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1204,"average_commit_lag_file_count":1,"start_time":1692320288397,"end_time":1692320320536,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:58:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: service closed +17-08-2023 17:58:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 +17-08-2023 17:58:40 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 is:0, names:[] +17-08-2023 17:58:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 dropped +17-08-2023 17:58:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 dropped +17-08-2023 17:58:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 +17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4898200611066290160 +17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 +17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:58:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:58:41 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:58:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:58:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, existing:[] +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 - service started +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4898200611066290160, topicPartition: test-0 +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4898200611066290160. +17-08-2023 17:58:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 retrieved 0 file names +17-08-2023 17:58:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160. +17-08-2023 17:58:42 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 doesn't exist +17-08-2023 17:58:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:58:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 retrieved 0 file names +17-08-2023 17:58:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4898200611066290160","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320321795} of type:kafka_pipe_start +17-08-2023 17:58:43 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: cleaner started +17-08-2023 17:58:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz +17-08-2023 17:58:43 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 17:58:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160, filePath:TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz +17-08-2023 17:58:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 +17-08-2023 17:58:43 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160, filePath:TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz +17-08-2023 17:58:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz +17-08-2023 17:58:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz] +17-08-2023 17:58:43 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz] +17-08-2023 17:58:43 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0/insertFiles?requestId=0da7d33f-b5d5-448b-9e43-a6ae4f502335&showSkippedFiles=false +17-08-2023 17:58:44 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:58:44 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574713348, X-Snowflake-Connection-Request: 1, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.410, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.19.132:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 17:58:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:58:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:58:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:59:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:59:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:59:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:59:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: cleaner terminated +17-08-2023 17:59:15 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 17:59:15 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 17:59:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 17:59:15 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 17:59:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4898200611066290160","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1114,"average_commit_lag_file_count":1,"start_time":1692320321794,"end_time":1692320355099,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 17:59:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: service closed +17-08-2023 17:59:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 +17-08-2023 17:59:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 is:0, names:[] +17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 dropped +17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 dropped +17-08-2023 17:59:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 17:59:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:16 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 17:59:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 17:59:16 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 17:59:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 17:59:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0, existing:[] +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 - service started +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4032550198220204688, topicPartition: test-0 +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 17:59:16 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: unknown bytes +17-08-2023 17:59:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 17:59:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4032550198220204688. +17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 retrieved 0 file names +17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688. +17-08-2023 17:59:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 doesn't exist +17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 +17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 retrieved 0 file names +17-08-2023 17:59:17 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0: cleaner started +17-08-2023 17:59:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4032550198220204688","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320356510} of type:kafka_pipe_start +17-08-2023 17:59:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_key_1692320357310.gz +17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_key_1692320357310.gz to table stage: kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_value_1692320357870.gz +17-08-2023 17:59:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_value_1692320357870.gz to table stage: kafka_connector_test_table_4032550198220204688 +17-08-2023 17:59:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:36 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:41 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:46 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 17:59:51 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 17:59:56 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:01 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:06 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:17 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4032550198220204688","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320356506,"end_time":1692320417323,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 18:00:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:00:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +17-08-2023 18:00:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:00:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 +17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8459553152922049676 +17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 +17-08-2023 18:00:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:00:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:00:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 +17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:44 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:00:45 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 18:00:45 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:00:45 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:00:45 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:00:45 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:45 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0, existing:[] +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 - service started +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8459553152922049676, topicPartition: test-0 +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8459553152922049676. +17-08-2023 18:00:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 retrieved 0 file names +17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676. +17-08-2023 18:00:45 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 doesn't exist +17-08-2023 18:00:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 +17-08-2023 18:00:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 retrieved 0 file names +17-08-2023 18:00:46 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0: cleaner started +17-08-2023 18:00:46 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8459553152922049676","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320445097} of type:kafka_pipe_start +17-08-2023 18:00:46 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8459553152922049676/0/0_0_1692320446358.json.gz +17-08-2023 18:00:46 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:01:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:01:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 +17-08-2023 18:01:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4764356504904401298 +17-08-2023 18:01:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:01:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 +17-08-2023 18:01:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:13 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:01:13 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 18:01:13 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:01:13 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:01:13 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:01:13 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:13 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, existing:[] +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 - service started +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4764356504904401298, topicPartition: test-0 +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4764356504904401298. +17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 retrieved 0 file names +17-08-2023 18:01:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298. +17-08-2023 18:01:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 doesn't exist +17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 retrieved 0 file names +17-08-2023 18:01:14 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: cleaner started +17-08-2023 18:01:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4764356504904401298","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320473586} of type:kafka_pipe_start +17-08-2023 18:01:14 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz +17-08-2023 18:01:14 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:01:14 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298, filePath:TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz +17-08-2023 18:01:14 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 +17-08-2023 18:01:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298, filePath:TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz +17-08-2023 18:01:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz +17-08-2023 18:01:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz] +17-08-2023 18:01:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz] +17-08-2023 18:01:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0/insertFiles?requestId=a3a4e95a-ace1-4ade-88ac-ef19de60009f&showSkippedFiles=false +17-08-2023 18:01:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:01:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574194690, X-Snowflake-Connection-Request: 92, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.510, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:01:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:18 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: cleaner terminated +17-08-2023 18:01:46 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:01:46 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:01:46 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:01:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:46 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4764356504904401298","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1682,"average_commit_lag_file_count":1,"start_time":1692320473578,"end_time":1692320506918,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:01:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: service closed +17-08-2023 18:01:46 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 +17-08-2023 18:01:46 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 is:0, names:[] +17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 dropped +17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 dropped +17-08-2023 18:01:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:01:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 +17-08-2023 18:01:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3495569082225838574 +17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:01:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 +17-08-2023 18:01:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:48 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:01:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:01:48 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:01:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:01:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:48 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, existing:[] +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 - service started +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3495569082225838574, topicPartition: test-0 +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:01:48 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 18:01:48 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3495569082225838574. +17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 retrieved 0 file names +17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574. +17-08-2023 18:01:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 doesn't exist +17-08-2023 18:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 retrieved 0 file names +17-08-2023 18:01:49 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320508306} of type:kafka_pipe_start +17-08-2023 18:01:49 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz +17-08-2023 18:01:49 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 18:01:49 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: cleaner started +17-08-2023 18:01:49 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","processed-offset":1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":178,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320508303,"end_time":1692320509267,"is_pipe_closing":false} of type:kafka_pipe_usage +17-08-2023 18:01:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574, filePath:TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz +17-08-2023 18:01:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 +17-08-2023 18:01:49 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574, filePath:TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz +17-08-2023 18:01:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz +17-08-2023 18:01:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz] +17-08-2023 18:01:49 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz] +17-08-2023 18:01:49 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0/insertFiles?requestId=e9ab114d-2391-4dbc-97e9-2bef7d42acaa&showSkippedFiles=false +17-08-2023 18:01:51 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:01:51 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574083877, X-Snowflake-Connection-Request: 114, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.555, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:01:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:01:53 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:01:58 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: cleaner terminated +17-08-2023 18:02:22 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:02:22 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:02:22 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:02:22 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2436,"average_commit_lag_file_count":1,"start_time":1692320509267,"end_time":1692320542991,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:02:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: service closed +17-08-2023 18:02:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 +17-08-2023 18:02:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 is:0, names:[] +17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 dropped +17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 dropped +17-08-2023 18:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 +17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 +17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7401106389637088935 +17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 +17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 +17-08-2023 18:02:24 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:02:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:02:24 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:02:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:02:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 +17-08-2023 18:02:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0, existing:[] +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 - service started +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7401106389637088935, topicPartition: test-0 +17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 dropped +17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 dropped +17-08-2023 18:02:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 +17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 +17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6415826046470952295 +17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 +17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 +17-08-2023 18:02:25 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:02:25 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:02:25 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:02:25 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 +17-08-2023 18:02:25 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0, existing:[] +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 - service started +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6415826046470952295, topicPartition: test-0 +17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 dropped +17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 dropped +17-08-2023 18:02:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:02:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 +17-08-2023 18:02:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6646659250009994864 +17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:02:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 +17-08-2023 18:02:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:27 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:02:27 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:02:27 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:02:27 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:27 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, existing:[] +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 - service started +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6646659250009994864, topicPartition: test-0 +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6646659250009994864. +17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 retrieved 0 file names +17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864. +17-08-2023 18:02:27 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 doesn't exist +17-08-2023 18:02:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:02:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 retrieved 0 file names +17-08-2023 18:02:28 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: cleaner started +17-08-2023 18:02:28 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6646659250009994864","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320547237} of type:kafka_pipe_start +17-08-2023 18:02:28 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz +17-08-2023 18:02:28 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:02:28 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864, filePath:TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz +17-08-2023 18:02:28 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 +17-08-2023 18:02:29 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864, filePath:TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz +17-08-2023 18:02:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz +17-08-2023 18:02:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz] +17-08-2023 18:02:29 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz] +17-08-2023 18:02:29 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0/insertFiles?requestId=bd47c693-89d9-4828-abc3-98b494db16f3&showSkippedFiles=false +17-08-2023 18:02:29 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 1; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 1; pending: 0; available: 0; max: 100] + +17-08-2023 18:02:29 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:02:29 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574786314, X-Snowflake-Connection-Request: 79, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.554, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:02:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:02:34 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:39 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:44 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:49 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:54 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:02:59 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: cleaner terminated +17-08-2023 18:03:00 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:03:00 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:03:00 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:03:00 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6646659250009994864","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1331,"average_commit_lag_file_count":1,"start_time":1692320547236,"end_time":1692320580786,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:03:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: service closed +17-08-2023 18:03:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 +17-08-2023 18:03:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 is:0, names:[] +17-08-2023 18:03:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 dropped +17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 dropped +17-08-2023 18:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 +17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3951058095210931105 +17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 +17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:01 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:03:01 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:01 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:01 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:03:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:01 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, existing:[] +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 - service started +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3951058095210931105, topicPartition: test-0 +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:03:01 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:03:01 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:02 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3951058095210931105. +17-08-2023 18:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 retrieved 0 file names +17-08-2023 18:03:02 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105. +17-08-2023 18:03:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 doesn't exist +17-08-2023 18:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 retrieved 0 file names +17-08-2023 18:03:03 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: cleaner started +17-08-2023 18:03:03 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3951058095210931105","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320581974} of type:kafka_pipe_start +17-08-2023 18:03:03 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz +17-08-2023 18:03:03 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 +17-08-2023 18:03:03 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105, filePath:TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz +17-08-2023 18:03:03 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 +17-08-2023 18:03:03 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105, filePath:TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz +17-08-2023 18:03:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz +17-08-2023 18:03:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz] +17-08-2023 18:03:03 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz] +17-08-2023 18:03:03 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0/insertFiles?requestId=377b4a1a-63ff-457a-a3ee-1316e3635a9c&showSkippedFiles=false +17-08-2023 18:03:04 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:03:04 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574388491, X-Snowflake-Connection-Request: 65, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.470, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:03:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:06 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: cleaner terminated +17-08-2023 18:03:35 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:03:35 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:03:35 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:03:35 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:35 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3951058095210931105","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1309,"average_commit_lag_file_count":1,"start_time":1692320581973,"end_time":1692320615528,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:03:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: service closed +17-08-2023 18:03:35 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 +17-08-2023 18:03:35 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 is:0, names:[] +17-08-2023 18:03:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 dropped +17-08-2023 18:03:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 dropped +17-08-2023 18:03:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 +17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 +17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2747719583120231178 +17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 +17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 +17-08-2023 18:03:36 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:36 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:03:36 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:36 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:03:36 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 +17-08-2023 18:03:36 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0, existing:[] +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 - service started +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2747719583120231178, topicPartition: test-0 +17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 dropped +17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 dropped +17-08-2023 18:03:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 +17-08-2023 18:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8941582472149256418 +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 +17-08-2023 18:03:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 +17-08-2023 18:03:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:38 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:03:38 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 +17-08-2023 18:03:38 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0, existing:[] +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 - service started +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8941582472149256418, topicPartition: test-0 +17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 dropped +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 dropped +17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:38 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:38 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:03:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:38 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:03:38 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:03:38 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:03:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +17-08-2023 18:03:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:03:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:03:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:03:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:03:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3921686411842415796 doesn't exist +17-08-2023 18:03:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3921686411842415796. +17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3921686411842415796 +17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:03:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:39 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:03:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:03:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:03:40 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +17-08-2023 18:03:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:03:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:03:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:03:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:03:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1041 +17-08-2023 18:03:40 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4613830906384878297 doesn't exist +17-08-2023 18:03:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4613830906384878297. +17-08-2023 18:03:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4613830906384878297 +17-08-2023 18:03:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +17-08-2023 18:03:41 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:03:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:41 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:03:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:03:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:03:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1041 +17-08-2023 18:03:41 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 +17-08-2023 18:03:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:03:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:03:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:03:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:03:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1645110627355484471 doesn't exist +17-08-2023 18:03:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1645110627355484471. +17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1645110627355484471 +17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:41 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:42 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:03:42 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:03:42 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:03:42 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:03:42 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:03:42 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:03:42 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:03:42 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1042 +17-08-2023 18:03:42 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 +17-08-2023 18:03:42 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:03:42 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:03:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:03:42 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:03:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1044 +17-08-2023 18:03:42 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_5970348261965741519 doesn't exist +17-08-2023 18:03:42 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_5970348261965741519. +17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5970348261965741519 +17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:10:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:10:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:10:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:10:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 +17-08-2023 18:10:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8312607821312555903 +17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:10:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 +17-08-2023 18:10:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:10:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 18:10:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:10:32 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:10:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:10:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, existing:[] +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 - service started +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8312607821312555903, topicPartition: test-0 +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8312607821312555903. +17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 retrieved 0 file names +17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903. +17-08-2023 18:10:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 doesn't exist +17-08-2023 18:10:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:10:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 retrieved 0 file names +17-08-2023 18:10:33 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: cleaner started +17-08-2023 18:10:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8312607821312555903","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321032333} of type:kafka_pipe_start +17-08-2023 18:10:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz +17-08-2023 18:10:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:10:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903, filePath:TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz +17-08-2023 18:10:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 +17-08-2023 18:10:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903, filePath:TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz +17-08-2023 18:10:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz +17-08-2023 18:10:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz] +17-08-2023 18:10:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz] +17-08-2023 18:10:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0/insertFiles?requestId=541da82e-1ae0-405f-ba62-be14c8674438&showSkippedFiles=false +17-08-2023 18:10:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:10:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575787957, X-Snowflake-Connection-Request: 448, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.497, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.12:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:10:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:10:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:10:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:10:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:10:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:10:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: cleaner terminated +17-08-2023 18:11:05 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:11:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:11:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:11:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8312607821312555903","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1537,"average_commit_lag_file_count":1,"start_time":1692321032326,"end_time":1692321065756,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:11:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: service closed +17-08-2023 18:11:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 +17-08-2023 18:11:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 is:0, names:[] +17-08-2023 18:11:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 dropped +17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 dropped +17-08-2023 18:11:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 +17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7267408819954433042 +17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 +17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:06 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:11:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:11:06 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:11:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:11:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, existing:[] +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 - service started +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7267408819954433042, topicPartition: test-0 +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:11:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +17-08-2023 18:11:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7267408819954433042. +17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 retrieved 0 file names +17-08-2023 18:11:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042. +17-08-2023 18:11:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 doesn't exist +17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 retrieved 0 file names +17-08-2023 18:11:07 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: cleaner started +17-08-2023 18:11:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7267408819954433042","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321066915} of type:kafka_pipe_start +17-08-2023 18:11:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz +17-08-2023 18:11:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 18:11:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042, filePath:TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz +17-08-2023 18:11:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 +17-08-2023 18:11:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042, filePath:TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz +17-08-2023 18:11:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz +17-08-2023 18:11:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz] +17-08-2023 18:11:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz] +17-08-2023 18:11:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0/insertFiles?requestId=e3d6c9d1-9a64-4c06-911a-cb696bf333b9&showSkippedFiles=false +17-08-2023 18:11:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:11:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574287154, X-Snowflake-Connection-Request: 170, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.364, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:11:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:26 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:36 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: cleaner terminated +17-08-2023 18:11:39 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:11:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:11:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:11:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7267408819954433042","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1270,"average_commit_lag_file_count":1,"start_time":1692321066913,"end_time":1692321099962,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:11:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: service closed +17-08-2023 18:11:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 +17-08-2023 18:11:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 is:0, names:[] +17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 dropped +17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 dropped +17-08-2023 18:11:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:11:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 +17-08-2023 18:11:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1282015380187074791 +17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:11:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 +17-08-2023 18:11:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:11:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:11:41 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:11:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:11:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, existing:[] +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 - service started +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1282015380187074791, topicPartition: test-0 +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1282015380187074791. +17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 retrieved 0 file names +17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791. +17-08-2023 18:11:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 doesn't exist +17-08-2023 18:11:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:11:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 retrieved 0 file names +17-08-2023 18:11:42 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: cleaner started +17-08-2023 18:11:42 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1282015380187074791","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321101234} of type:kafka_pipe_start +17-08-2023 18:11:42 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz +17-08-2023 18:11:42 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:11:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791, filePath:TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz +17-08-2023 18:11:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 +17-08-2023 18:11:42 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791, filePath:TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz +17-08-2023 18:11:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz +17-08-2023 18:11:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz] +17-08-2023 18:11:42 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz] +17-08-2023 18:11:42 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0/insertFiles?requestId=df06f6d4-2b62-4bfc-b7ba-2069e9db3a93&showSkippedFiles=false +17-08-2023 18:11:43 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:11:43 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574527671, X-Snowflake-Connection-Request: 159, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.428, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:11:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:11:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:11:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: cleaner terminated +17-08-2023 18:12:14 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:12:14 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:12:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:12:14 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1282015380187074791","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1115,"average_commit_lag_file_count":1,"start_time":1692321101230,"end_time":1692321134049,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:12:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: service closed +17-08-2023 18:12:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 +17-08-2023 18:12:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 is:0, names:[] +17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 dropped +17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 dropped +17-08-2023 18:12:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:12:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 +17-08-2023 18:12:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6790576732472386043 +17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:12:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 +17-08-2023 18:12:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:15 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:12:15 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:12:15 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:12:15 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:12:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:15 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, existing:[] +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 - service started +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6790576732472386043, topicPartition: test-0 +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:12:15 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:12:15 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6790576732472386043. +17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 retrieved 0 file names +17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043. +17-08-2023 18:12:15 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 doesn't exist +17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 retrieved 0 file names +17-08-2023 18:12:16 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6790576732472386043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321135261} of type:kafka_pipe_start +17-08-2023 18:12:16 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: cleaner started +17-08-2023 18:12:16 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz +17-08-2023 18:12:16 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 18:12:16 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043, filePath:TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz +17-08-2023 18:12:16 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 +17-08-2023 18:12:16 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043, filePath:TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz +17-08-2023 18:12:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz +17-08-2023 18:12:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz] +17-08-2023 18:12:16 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz] +17-08-2023 18:12:16 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0/insertFiles?requestId=20398aef-d45c-4766-934d-3e29876e90ec&showSkippedFiles=false +17-08-2023 18:12:17 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:12:17 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574456708, X-Snowflake-Connection-Request: 66, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.508, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.12:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:12:17 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:20 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:25 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:30 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:35 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:40 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:45 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: cleaner terminated +17-08-2023 18:12:48 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:12:48 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:12:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:12:48 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:48 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6790576732472386043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1249,"average_commit_lag_file_count":1,"start_time":1692321135238,"end_time":1692321168243,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:12:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: service closed +17-08-2023 18:12:48 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 +17-08-2023 18:12:48 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 is:0, names:[] +17-08-2023 18:12:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 dropped +17-08-2023 18:12:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 dropped +17-08-2023 18:12:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 +17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:49 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2269349621436228672 +17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 +17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:49 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:12:49 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:12:49 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:12:49 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:12:49 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:49 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, existing:[] +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 - service started +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2269349621436228672, topicPartition: test-0 +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2269349621436228672. +17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 retrieved 0 file names +17-08-2023 18:12:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672. +17-08-2023 18:12:50 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 doesn't exist +17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 retrieved 0 file names +17-08-2023 18:12:50 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: cleaner started +17-08-2023 18:12:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2269349621436228672","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321169653} of type:kafka_pipe_start +17-08-2023 18:12:50 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz +17-08-2023 18:12:50 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:12:50 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672, filePath:TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz +17-08-2023 18:12:50 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 +17-08-2023 18:12:51 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672, filePath:TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz +17-08-2023 18:12:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz +17-08-2023 18:12:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz] +17-08-2023 18:12:51 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz] +17-08-2023 18:12:51 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0/insertFiles?requestId=af7408a7-5401-4c89-b0c0-fd6f213e25dc&showSkippedFiles=false +17-08-2023 18:12:51 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:12:51 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574301308, X-Snowflake-Connection-Request: 341, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.410, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.19.45:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:12:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:12:54 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:12:59 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:04 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:09 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:14 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:19 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: cleaner terminated +17-08-2023 18:13:22 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:13:22 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:13:22 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:13:22 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2269349621436228672","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1253,"average_commit_lag_file_count":1,"start_time":1692321169652,"end_time":1692321202791,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:13:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: service closed +17-08-2023 18:13:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 +17-08-2023 18:13:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 is:0, names:[] +17-08-2023 18:13:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 dropped +17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 dropped +17-08-2023 18:13:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:13:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 +17-08-2023 18:13:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:23 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9008642977022007506 +17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:13:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 +17-08-2023 18:13:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:24 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:13:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:13:24 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:13:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:13:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, existing:[] +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 - service started +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_9008642977022007506, topicPartition: test-0 +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:13:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:13:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_9008642977022007506. +17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 retrieved 0 file names +17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506. +17-08-2023 18:13:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 doesn't exist +17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 retrieved 0 file names +17-08-2023 18:13:24 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: cleaner started +17-08-2023 18:13:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_9008642977022007506","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321204019} of type:kafka_pipe_start +17-08-2023 18:13:24 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz +17-08-2023 18:13:24 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 +17-08-2023 18:13:24 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506, filePath:TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz +17-08-2023 18:13:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 +17-08-2023 18:13:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506, filePath:TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz +17-08-2023 18:13:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz +17-08-2023 18:13:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz] +17-08-2023 18:13:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz] +17-08-2023 18:13:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0/insertFiles?requestId=f3fda864-d919-421b-9691-aaf735e677bd&showSkippedFiles=false +17-08-2023 18:13:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:13:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574947967, X-Snowflake-Connection-Request: 441, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.462, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.237:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:13:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:29 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:34 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:39 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:44 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:49 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:54 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: cleaner terminated +17-08-2023 18:13:57 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:13:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:13:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:13:57 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:13:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_9008642977022007506","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1206,"average_commit_lag_file_count":1,"start_time":1692321204018,"end_time":1692321237136,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:13:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: service closed +17-08-2023 18:13:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 +17-08-2023 18:13:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 is:0, names:[] +17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 dropped +17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 dropped +17-08-2023 18:13:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:13:57 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 +17-08-2023 18:13:57 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_603667045474338019 +17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:13:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 +17-08-2023 18:13:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:58 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:13:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:13:58 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:13:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:13:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, existing:[] +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 - service started +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_603667045474338019, topicPartition: test-0 +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_603667045474338019. +17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 retrieved 0 file names +17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019. +17-08-2023 18:13:58 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 doesn't exist +17-08-2023 18:13:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:13:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 retrieved 0 file names +17-08-2023 18:13:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_603667045474338019","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321238382} of type:kafka_pipe_start +17-08-2023 18:13:59 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: cleaner started +17-08-2023 18:13:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz +17-08-2023 18:13:59 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +17-08-2023 18:13:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019, filePath:TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz +17-08-2023 18:13:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 +17-08-2023 18:13:59 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019, filePath:TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz +17-08-2023 18:13:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz +17-08-2023 18:13:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz] +17-08-2023 18:13:59 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz] +17-08-2023 18:13:59 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0/insertFiles?requestId=a9eeb257-29c2-415b-8fcb-37a8ea09a6c9&showSkippedFiles=false +17-08-2023 18:14:00 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:14:00 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575854182, X-Snowflake-Connection-Request: 274, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.370, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.21.27:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:14:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:03 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:08 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:13 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:18 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:23 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:28 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: cleaner terminated +17-08-2023 18:14:31 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:14:31 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:14:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:14:31 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_603667045474338019","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1215,"average_commit_lag_file_count":1,"start_time":1692321238381,"end_time":1692321271254,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:14:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: service closed +17-08-2023 18:14:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 +17-08-2023 18:14:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 is:0, names:[] +17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 dropped +17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 dropped +17-08-2023 18:14:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 +17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6269835192143780681 +17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 +17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:32 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:14:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:14:32 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:14:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +17-08-2023 18:14:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, existing:[] +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 - service started +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6269835192143780681, topicPartition: test-0 +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:14:32 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:14:32 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6269835192143780681. +17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 retrieved 0 file names +17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681. +17-08-2023 18:14:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 doesn't exist +17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 retrieved 0 file names +17-08-2023 18:14:33 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: cleaner started +17-08-2023 18:14:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6269835192143780681","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321272495} of type:kafka_pipe_start +17-08-2023 18:14:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz +17-08-2023 18:14:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +17-08-2023 18:14:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681, filePath:TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz +17-08-2023 18:14:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 +17-08-2023 18:14:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681, filePath:TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz +17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz +17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz] +17-08-2023 18:14:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz] +17-08-2023 18:14:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0/insertFiles?requestId=52c59c2f-4302-48b0-acb3-18298072b2ef&showSkippedFiles=false +17-08-2023 18:14:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:14:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575126666, X-Snowflake-Connection-Request: 40, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.749, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.53:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +17-08-2023 18:14:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:14:37 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:42 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:47 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:52 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:14:57 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:15:02 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:15:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: cleaner terminated +17-08-2023 18:15:05 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +17-08-2023 18:15:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:15:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +17-08-2023 18:15:05 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:15:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6269835192143780681","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1468,"average_commit_lag_file_count":1,"start_time":1692321272493,"end_time":1692321305951,"is_pipe_closing":true} of type:kafka_pipe_usage +17-08-2023 18:15:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: service closed +17-08-2023 18:15:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 +17-08-2023 18:15:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 is:0, names:[] +17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 dropped +17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 dropped +17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:06 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:15:06 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:15:06 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:15:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:06 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:15:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:15:06 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:15:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:15:06 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:15:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:15:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +17-08-2023 18:15:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:15:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:15:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:15:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:15:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:15:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3026528099495733147 doesn't exist +17-08-2023 18:15:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3026528099495733147. +17-08-2023 18:15:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3026528099495733147 +17-08-2023 18:15:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:15:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:15:08 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:15:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:15:08 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:15:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:15:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:15:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:15:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +17-08-2023 18:15:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:15:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_8031857255777630677 doesn't exist +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_8031857255777630677. +17-08-2023 18:15:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1050 +17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8031857255777630677 +17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:15:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:15:08 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:15:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:15:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:15:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:15:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 +17-08-2023 18:15:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:15:09 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4437478255481855804 doesn't exist +17-08-2023 18:15:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4437478255481855804. +17-08-2023 18:15:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +17-08-2023 18:15:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1049 +17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4437478255481855804 +17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:09 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:15:09 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:15:09 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:09 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:15:09 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:15:09 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:15:09 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:15:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:15:09 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 +17-08-2023 18:15:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:15:09 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:15:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:15:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:15:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1050, currentDiffMillis=1050 +17-08-2023 18:15:10 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6505219357856853769 doesn't exist +17-08-2023 18:15:10 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6505219357856853769. +17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1047 +17-08-2023 18:15:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6505219357856853769 +17-08-2023 18:15:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 +17-08-2023 18:16:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:16:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:16:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:16:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:16:35 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:16:35 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:16:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:16:35 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:16:35 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:16:35 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +17-08-2023 18:16:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:16:35 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:16:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:16:35 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:16:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:16:35 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +17-08-2023 18:16:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:16:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:16:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:16:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:16:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:16:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1722213013389090901 doesn't exist +17-08-2023 18:16:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1722213013389090901. +17-08-2023 18:16:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1722213013389090901 +17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1722213013389090901 +17-08-2023 18:16:36 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1722213013389090901, client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:16:36 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_0","table":"kafka_connector_test_table_1722213013389090901"} +17-08-2023 18:16:36 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1722213013389090901, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:16:36 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901 +17-08-2023 18:16:36 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:36 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset:null +17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +17-08-2023 18:16:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1722213013389090901, topicPartition: test-0 +17-08-2023 18:16:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +17-08-2023 18:16:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:16:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1049 +17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 +17-08-2023 18:16:36 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1ade3eba[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec] +17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, idx=0 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. +17-08-2023 18:16:36 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +17-08-2023 18:16:36 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1104: footer length = 754 +17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 754 => 242 2 0 0 +17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1112, encryptedCompressedSize=1120, bdecVersion=THREE +17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, size=1120 +17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, size=1120, timeInMillis=460 +17-08-2023 18:16:37 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, idx=0 +17-08-2023 18:16:37 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 +17-08-2023 18:16:37 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +17-08-2023 18:16:37 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec","md5":"acf9ad2c9a58b0c7e5ebfdf72bf2ebb5","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","chunk_start_offset":0,"chunk_length":1112,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"e18d3c69c900cd5479811d18a15859ff","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370132722,"first_insert_time_in_ms":1692321396716,"last_insert_time_in_ms":1692321396716,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321396880,"build_duration_ms":229,"upload_duration_ms":458},"bdec_version":3}],"role":"testrole_kafka","request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_2"} +17-08-2023 18:16:37 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +17-08-2023 18:16:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +17-08-2023 18:16:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1050 +17-08-2023 18:16:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 +17-08-2023 18:16:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:16:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +17-08-2023 18:16:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:16:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +17-08-2023 18:16:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 +17-08-2023 18:16:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1052 +17-08-2023 18:16:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:16:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:16:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 +17-08-2023 18:16:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 +17-08-2023 18:16:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +17-08-2023 18:16:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:16:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:16:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +17-08-2023 18:16:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +17-08-2023 18:16:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +17-08-2023 18:16:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +17-08-2023 18:16:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:16:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:16:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1038 +17-08-2023 18:16:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +17-08-2023 18:16:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +17-08-2023 18:16:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:16:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +17-08-2023 18:16:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +17-08-2023 18:17:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +17-08-2023 18:17:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +17-08-2023 18:17:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +17-08-2023 18:17:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:17:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +17-08-2023 18:17:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +17-08-2023 18:17:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 +17-08-2023 18:17:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset:0 +17-08-2023 18:17:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +17-08-2023 18:17:07 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, channel sequencer=0, row sequencer=1 +17-08-2023 18:17:07 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:08 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +17-08-2023 18:17:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=958, currentDiffMillis=958 +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=311, min=0, max=41, mean=0.46869509192368386, stddev=2.9561123487373013, p50=0.0, p75=0.0, p95=1.0, p98=11.0, p99=16.0, p999=41.0 +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.3980447542096552, rate_unit=events/second +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=34.43323915113383, rate_unit=events/second +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=229.239667, max=229.239667, mean=229.239667, stddev=0.0, p50=229.239667, p75=229.239667, p95=229.239667, p98=229.239667, p99=229.239667, p999=229.239667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074203691484075, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=860.656333, max=860.656333, mean=860.656333, stddev=0.0, p50=860.656333, p75=860.656333, p95=860.656333, p98=860.656333, p99=860.656333, p999=860.656333, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.030741800766580445, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=168.713084, max=168.713084, mean=168.713084, stddev=0.0, p50=168.713084, p75=168.713084, p95=168.713084, p98=168.713084, p99=168.713084, p999=168.713084, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074155725816862, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=458.591583, max=458.591583, mean=458.591583, stddev=0.0, p50=458.591583, p75=458.591583, p95=458.591583, p98=458.591583, p99=458.591583, p999=458.591583, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074134131822672, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:08 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:17:08 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +17-08-2023 18:17:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:17:08 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +17-08-2023 18:17:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:08 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:17:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:17:08 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:17:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:17:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:17:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:17:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:17:09 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_7960705935937769956 doesn't exist +17-08-2023 18:17:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_7960705935937769956. +17-08-2023 18:17:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7960705935937769956 +17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_7960705935937769956 +17-08-2023 18:17:09 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7960705935937769956, client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_0","table":"kafka_connector_test_table_7960705935937769956"} +17-08-2023 18:17:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7960705935937769956, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:09 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956 +17-08-2023 18:17:09 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset:null +17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +17-08-2023 18:17:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7960705935937769956, topicPartition: test-0 +17-08-2023 18:17:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:17:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:17:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} +17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 +17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +17-08-2023 18:17:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 +17-08-2023 18:17:09 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@627a88de[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, idx=0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1155: footer length = 775 +17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 775 => 7 3 0 0 +17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1163, encryptedCompressedSize=1168, bdecVersion=THREE +17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, size=1168 +17-08-2023 18:17:10 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, size=1168, timeInMillis=359 +17-08-2023 18:17:10 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, idx=0 +17-08-2023 18:17:10 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 +17-08-2023 18:17:10 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +17-08-2023 18:17:10 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec","md5":"e530356d0ad4914ffc1892c49b67d004","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","chunk_start_offset":0,"chunk_length":1163,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"73d9563412656d766f9c49ce85954ee5","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370136334,"first_insert_time_in_ms":1692321429613,"last_insert_time_in_ms":1692321429613,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321429947,"build_duration_ms":14,"upload_duration_ms":358},"bdec_version":3}],"role":"testrole_kafka","request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_2"} +17-08-2023 18:17:10 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +17-08-2023 18:17:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +17-08-2023 18:17:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +17-08-2023 18:17:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +17-08-2023 18:17:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +17-08-2023 18:17:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:17:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:17:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:17:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +17-08-2023 18:17:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +17-08-2023 18:17:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +17-08-2023 18:17:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +17-08-2023 18:17:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:17:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +17-08-2023 18:17:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +17-08-2023 18:17:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +17-08-2023 18:17:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +17-08-2023 18:17:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +17-08-2023 18:17:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:17:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1030 +17-08-2023 18:17:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +17-08-2023 18:17:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +17-08-2023 18:17:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +17-08-2023 18:17:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:33 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +17-08-2023 18:17:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +17-08-2023 18:17:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +17-08-2023 18:17:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +17-08-2023 18:17:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +17-08-2023 18:17:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:38 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +17-08-2023 18:17:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +17-08-2023 18:17:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 +17-08-2023 18:17:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset:1 +17-08-2023 18:17:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, channel sequencer=0, row sequencer=1 +17-08-2023 18:17:40 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:40 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +17-08-2023 18:17:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +17-08-2023 18:17:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=806, currentDiffMillis=806 +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=10, mean=0.14629850214578627, stddev=0.8703954391484999, p50=0.0, p75=0.0, p95=1.0, p98=2.0, p99=6.0, p999=10.0 +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.588906402148592, rate_unit=events/second +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.43167012569987, rate_unit=events/second +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=14.945459, max=14.945459, mean=14.945459, stddev=0.0, p50=14.945459, p75=14.945459, p95=14.945459, p98=14.945459, p99=14.945459, p999=14.945459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119125744053028, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=560.997667, max=560.997667, mean=560.997667, stddev=0.0, p50=560.997667, p75=560.997667, p95=560.997667, p98=560.997667, p99=560.997667, p999=560.997667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119105155210925, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=184.702625, max=184.702625, mean=184.702625, stddev=0.0, p50=184.702625, p75=184.702625, p95=184.702625, p98=184.702625, p99=184.702625, p999=184.702625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031190884338964015, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=358.91875, max=358.91875, mean=358.91875, stddev=0.0, p50=358.91875, p75=358.91875, p95=358.91875, p98=358.91875, p99=358.91875, p999=358.91875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119069090125127, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:17:40 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:17:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +17-08-2023 18:17:40 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:41 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:17:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:17:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:17:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:17:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:17:41 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:17:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:17:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:17:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:17:41 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 +17-08-2023 18:17:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:17:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:17:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:17:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:17:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:17:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4166363975679526244 doesn't exist +17-08-2023 18:17:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4166363975679526244. +17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4166363975679526244 +17-08-2023 18:17:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4166363975679526244 +17-08-2023 18:17:41 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4166363975679526244, client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:17:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_0","table":"kafka_connector_test_table_4166363975679526244"} +17-08-2023 18:17:42 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4166363975679526244, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:17:42 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244 +17-08-2023 18:17:42 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:42 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset:null +17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +17-08-2023 18:17:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4166363975679526244, topicPartition: test-0 +17-08-2023 18:17:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +17-08-2023 18:17:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:17:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 +17-08-2023 18:17:42 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@655305a[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, idx=0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1104: footer length = 754 +17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 754 => 242 2 0 0 +17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1112, encryptedCompressedSize=1120, bdecVersion=THREE +17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, size=1120 +17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, size=1120, timeInMillis=241 +17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, idx=0 +17-08-2023 18:17:42 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 +17-08-2023 18:17:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +17-08-2023 18:17:42 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec","md5":"917a0ad4ef72db2517f75b653c7ee6a7","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","chunk_start_offset":0,"chunk_length":1112,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"b11b3389c80bd9ed979cc8292da58dd7","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370137822,"first_insert_time_in_ms":1692321462313,"last_insert_time_in_ms":1692321462313,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321462644,"build_duration_ms":11,"upload_duration_ms":241},"bdec_version":3}],"role":"testrole_kafka","request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_2"} +17-08-2023 18:17:43 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +17-08-2023 18:17:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +17-08-2023 18:17:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1020, currentDiffMillis=1021 +17-08-2023 18:17:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1015, currentDiffMillis=1015 +17-08-2023 18:17:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:17:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +17-08-2023 18:17:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +17-08-2023 18:17:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:17:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:17:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +17-08-2023 18:17:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +17-08-2023 18:17:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +17-08-2023 18:17:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +17-08-2023 18:17:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +17-08-2023 18:17:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:17:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:17:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +17-08-2023 18:17:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:17:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +17-08-2023 18:17:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +17-08-2023 18:18:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +17-08-2023 18:18:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1047 +17-08-2023 18:18:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 +17-08-2023 18:18:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:18:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +17-08-2023 18:18:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:18:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +17-08-2023 18:18:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +17-08-2023 18:18:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +17-08-2023 18:18:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +17-08-2023 18:18:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +17-08-2023 18:18:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +17-08-2023 18:18:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 +17-08-2023 18:18:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:18:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset:0 +17-08-2023 18:18:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, channel sequencer=0, row sequencer=1 +17-08-2023 18:18:13 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:18:13 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +17-08-2023 18:18:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +17-08-2023 18:18:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=757, currentDiffMillis=758 +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=6, mean=0.07622377515040273, stddev=0.46354309399795246, p50=0.0, p75=0.0, p95=0.0, p98=2.0, p99=2.0, p999=6.0 +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.439352093933107, rate_unit=events/second +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.026384399007306, rate_unit=events/second +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=11.616833, max=11.616833, mean=11.616833, stddev=0.0, p50=11.616833, p75=11.616833, p95=11.616833, p98=11.616833, p99=11.616833, p999=11.616833, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127327004150369, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=435.93925, max=435.93925, mean=435.93925, stddev=0.0, p50=435.93925, p75=435.93925, p95=435.93925, p98=435.93925, p99=435.93925, p999=435.93925, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127301771504547, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=181.473708, max=181.473708, mean=181.473708, stddev=0.0, p50=181.473708, p75=181.473708, p95=181.473708, p98=181.473708, p99=181.473708, p999=181.473708, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127282549782918, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=241.212625, max=241.212625, mean=241.212625, stddev=0.0, p50=241.212625, p75=241.212625, p95=241.212625, p98=241.212625, p99=241.212625, p999=241.212625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031272494777512684, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:13 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:18:13 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:18:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +17-08-2023 18:18:13 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:13 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +17-08-2023 18:18:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +17-08-2023 18:18:14 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +17-08-2023 18:18:14 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +17-08-2023 18:18:14 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:14 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +17-08-2023 18:18:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +17-08-2023 18:18:14 main INFO SecurityManager:190 - Successfully created new JWT +17-08-2023 18:18:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +17-08-2023 18:18:14 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +17-08-2023 18:18:14 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 +17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +17-08-2023 18:18:14 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +17-08-2023 18:18:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6835116767498869672 doesn't exist +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6835116767498869672. +17-08-2023 18:18:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6835116767498869672 +17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_6835116767498869672 +17-08-2023 18:18:14 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6835116767498869672, client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_0","table":"kafka_connector_test_table_6835116767498869672"} +17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6835116767498869672, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672 +17-08-2023 18:18:14 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset:null +17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6835116767498869672, topicPartition: test-0 +17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +17-08-2023 18:18:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +17-08-2023 18:18:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} +17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 +17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +17-08-2023 18:18:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 +17-08-2023 18:18:15 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@37654ab6[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, enableParquetMemoryOptimization=false +17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, idx=0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1112: footer length = 750 +17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 750 => 238 2 0 0 +17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1120, encryptedCompressedSize=1136, bdecVersion=THREE +17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, size=1136 +17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, size=1136, timeInMillis=249 +17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, idx=0 +17-08-2023 18:18:15 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 +17-08-2023 18:18:15 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +17-08-2023 18:18:15 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec","md5":"bc9ff0cc0a667cd1922c134e1dc49a48","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","chunk_start_offset":0,"chunk_length":1120,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"9710545448fc26ceb55354751239a1a1","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370141602,"first_insert_time_in_ms":1692321494973,"last_insert_time_in_ms":1692321494973,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321495254,"build_duration_ms":11,"upload_duration_ms":248},"bdec_version":3}],"role":"testrole_kafka","request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_2"} +17-08-2023 18:18:15 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +17-08-2023 18:18:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1025 +17-08-2023 18:18:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +17-08-2023 18:18:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +17-08-2023 18:18:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +17-08-2023 18:18:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +17-08-2023 18:18:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +17-08-2023 18:18:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +17-08-2023 18:18:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1026 +17-08-2023 18:18:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +17-08-2023 18:18:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +17-08-2023 18:18:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +17-08-2023 18:18:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +17-08-2023 18:18:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:18:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +17-08-2023 18:18:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +17-08-2023 18:18:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +17-08-2023 18:18:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +17-08-2023 18:18:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +17-08-2023 18:18:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +17-08-2023 18:18:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +17-08-2023 18:18:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +17-08-2023 18:18:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +17-08-2023 18:18:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:18:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +17-08-2023 18:18:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 +17-08-2023 18:18:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +17-08-2023 18:18:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1024 +17-08-2023 18:18:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +17-08-2023 18:18:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +17-08-2023 18:18:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +17-08-2023 18:18:46 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 +17-08-2023 18:18:46 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:18:46 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset:1 +17-08-2023 18:18:46 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, channel sequencer=0, row sequencer=1 +17-08-2023 18:18:46 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +17-08-2023 18:18:46 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +17-08-2023 18:18:46 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +17-08-2023 18:18:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=942, currentDiffMillis=943 +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=311, min=0, max=13, mean=0.2378348527545219, stddev=1.2813385457487345, p50=0.0, p75=0.0, p95=1.0, p98=5.0, p99=8.0, p999=13.0 +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.3896226230756414, rate_unit=events/second +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.25453620654256, rate_unit=events/second +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=11.89775, max=11.89775, mean=11.89775, stddev=0.0, p50=11.89775, p75=11.89775, p95=11.89775, p98=11.89775, p99=11.89775, p999=11.89775, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033665007850115, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=431.422375, max=431.422375, mean=431.422375, stddev=0.0, p50=431.422375, p75=431.422375, p95=431.422375, p98=431.422375, p99=431.422375, p999=431.422375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033466453044868, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=169.58025, max=169.58025, mean=169.58025, stddev=0.0, p50=169.58025, p75=169.58025, p95=169.58025, p98=169.58025, p99=169.58025, p999=169.58025, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03103331677537786, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=248.38375, max=248.38375, mean=248.38375, stddev=0.0, p50=248.38375, p75=248.38375, p95=248.38375, p98=248.38375, p99=248.38375, p999=248.38375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033147356489293, rate_unit=events/second, duration_unit=milliseconds +17-08-2023 18:18:46 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:46 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +17-08-2023 18:18:46 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +17-08-2023 18:18:46 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +17-08-2023 18:18:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:39:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:39:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 +18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4574998329344794831 +18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 +18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:39:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 12:39:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:39:32 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:39:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:39:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, existing:[] +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 - service started +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4574998329344794831, topicPartition: test-0 +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4574998329344794831. +18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 retrieved 0 file names +18-08-2023 12:39:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831. +18-08-2023 12:39:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 doesn't exist +18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 retrieved 0 file names +18-08-2023 12:39:33 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: cleaner started +18-08-2023 12:39:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4574998329344794831","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387572739} of type:kafka_pipe_start +18-08-2023 12:39:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz +18-08-2023 12:39:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 12:39:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831, filePath:TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz +18-08-2023 12:39:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 +18-08-2023 12:39:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831, filePath:TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz +18-08-2023 12:39:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz +18-08-2023 12:39:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz] +18-08-2023 12:39:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz] +18-08-2023 12:39:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0/insertFiles?requestId=bf5cd097-b311-42ab-b339-6bb58f641558&showSkippedFiles=false +18-08-2023 12:39:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:39:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:39:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:39:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:39:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:39:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:39:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:39:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: cleaner terminated +18-08-2023 12:40:05 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:40:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:40:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:40:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4574998329344794831","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1291,"average_commit_lag_file_count":1,"start_time":1692387572727,"end_time":1692387605673,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:40:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: service closed +18-08-2023 12:40:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 +18-08-2023 12:40:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 is:0, names:[] +18-08-2023 12:40:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 dropped +18-08-2023 12:40:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 dropped +18-08-2023 12:40:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 +18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3789295266337469155 +18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 +18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:06 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:40:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:40:06 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:40:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:40:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, existing:[] +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 - service started +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3789295266337469155, topicPartition: test-0 +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:40:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 12:40:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3789295266337469155. +18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 retrieved 0 file names +18-08-2023 12:40:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155. +18-08-2023 12:40:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 doesn't exist +18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 retrieved 0 file names +18-08-2023 12:40:07 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: cleaner started +18-08-2023 12:40:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3789295266337469155","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387606758} of type:kafka_pipe_start +18-08-2023 12:40:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz +18-08-2023 12:40:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 12:40:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155, filePath:TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz +18-08-2023 12:40:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 +18-08-2023 12:40:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155, filePath:TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz +18-08-2023 12:40:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz +18-08-2023 12:40:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz] +18-08-2023 12:40:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz] +18-08-2023 12:40:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0/insertFiles?requestId=3797ef7f-5ebf-4165-bb3f-ff0a94265c55&showSkippedFiles=false +18-08-2023 12:40:08 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:40:08 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:40:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:26 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:36 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: cleaner terminated +18-08-2023 12:40:39 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:40:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:40:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:40:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3789295266337469155","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1163,"average_commit_lag_file_count":1,"start_time":1692387606757,"end_time":1692387639612,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:40:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: service closed +18-08-2023 12:40:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 +18-08-2023 12:40:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 is:0, names:[] +18-08-2023 12:40:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 dropped +18-08-2023 12:40:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 dropped +18-08-2023 12:40:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 +18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7746804880075842119 +18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 +18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:40 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:40:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:40:40 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:40:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:40:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:40 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, existing:[] +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 - service started +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7746804880075842119, topicPartition: test-0 +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7746804880075842119. +18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 retrieved 0 file names +18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119. +18-08-2023 12:40:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 doesn't exist +18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 retrieved 0 file names +18-08-2023 12:40:41 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: cleaner started +18-08-2023 12:40:41 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7746804880075842119","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387640571} of type:kafka_pipe_start +18-08-2023 12:40:41 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz +18-08-2023 12:40:41 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 12:40:41 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119, filePath:TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz +18-08-2023 12:40:41 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 +18-08-2023 12:40:41 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119, filePath:TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz +18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz +18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz] +18-08-2023 12:40:41 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz] +18-08-2023 12:40:41 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0/insertFiles?requestId=d865cd6b-f441-461f-8d12-04745d7a350a&showSkippedFiles=false +18-08-2023 12:40:42 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:40:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:40:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:40:45 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:50 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:40:55 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:00 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:05 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:10 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: cleaner terminated +18-08-2023 12:41:13 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:41:13 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:41:13 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:41:13 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:13 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7746804880075842119","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1049,"average_commit_lag_file_count":1,"start_time":1692387640562,"end_time":1692387673089,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:41:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: service closed +18-08-2023 12:41:13 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 +18-08-2023 12:41:13 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 is:0, names:[] +18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 dropped +18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 dropped +18-08-2023 12:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 +18-08-2023 12:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2936916385938205034 +18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:41:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 +18-08-2023 12:41:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:14 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:41:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:41:14 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:41:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:41:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:14 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, existing:[] +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 - service started +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2936916385938205034, topicPartition: test-0 +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:41:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 12:41:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2936916385938205034. +18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 retrieved 0 file names +18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034. +18-08-2023 12:41:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 doesn't exist +18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 retrieved 0 file names +18-08-2023 12:41:15 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: cleaner started +18-08-2023 12:41:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2936916385938205034","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387674196} of type:kafka_pipe_start +18-08-2023 12:41:15 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz +18-08-2023 12:41:15 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 12:41:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034, filePath:TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz +18-08-2023 12:41:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 +18-08-2023 12:41:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034, filePath:TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz +18-08-2023 12:41:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz +18-08-2023 12:41:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz] +18-08-2023 12:41:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz] +18-08-2023 12:41:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0/insertFiles?requestId=2f41ab2f-9601-4784-a693-bf327aedfd1c&showSkippedFiles=false +18-08-2023 12:41:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:41:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: cleaner terminated +18-08-2023 12:41:47 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:41:47 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:41:47 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:41:47 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:47 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2936916385938205034","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1015,"average_commit_lag_file_count":1,"start_time":1692387674175,"end_time":1692387707047,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:41:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: service closed +18-08-2023 12:41:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 +18-08-2023 12:41:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 is:0, names:[] +18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 dropped +18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 dropped +18-08-2023 12:41:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:41:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 +18-08-2023 12:41:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:47 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4529001433918421359 +18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:41:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 +18-08-2023 12:41:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:48 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:41:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:41:48 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:41:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:41:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:48 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, existing:[] +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 - service started +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4529001433918421359, topicPartition: test-0 +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4529001433918421359. +18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 retrieved 0 file names +18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359. +18-08-2023 12:41:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 doesn't exist +18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 retrieved 0 file names +18-08-2023 12:41:48 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: cleaner started +18-08-2023 12:41:48 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4529001433918421359","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387708039} of type:kafka_pipe_start +18-08-2023 12:41:48 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz +18-08-2023 12:41:48 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 12:41:48 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359, filePath:TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz +18-08-2023 12:41:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 +18-08-2023 12:41:49 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359, filePath:TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz +18-08-2023 12:41:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz +18-08-2023 12:41:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz] +18-08-2023 12:41:49 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz] +18-08-2023 12:41:49 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0/insertFiles?requestId=683d72b3-2346-47f0-a506-995cc115d6b1&showSkippedFiles=false +18-08-2023 12:41:49 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:49 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:41:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:53 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_1 +18-08-2023 12:41:56 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:41:56 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:41:56 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_191448210870743606 +18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:41:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:41:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 +18-08-2023 12:41:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 +18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:56 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:41:57 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 12:41:57 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:41:57 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:41:57 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:41:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:57 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, existing:[] +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 - service started +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_191448210870743606, topicPartition: test-0 +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_191448210870743606. +18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names +18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606. +18-08-2023 12:41:57 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 doesn't exist +18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names +18-08-2023 12:41:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: cleaner started +18-08-2023 12:41:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387717118} of type:kafka_pipe_start +18-08-2023 12:41:57 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz +18-08-2023 12:41:57 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 3 records, 7136 bytes, offset 0 - 2 +18-08-2023 12:41:58 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:41:58 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606, filePath:TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz +18-08-2023 12:41:58 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:41:58 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606, filePath:TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz +18-08-2023 12:41:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz +18-08-2023 12:41:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names +18-08-2023 12:41:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] +18-08-2023 12:41:58 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] +18-08-2023 12:41:58 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0/insertFiles?requestId=5e0180ed-bc83-4a7a-80c4-965e4fe8a9aa&showSkippedFiles=false +18-08-2023 12:41:59 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:59 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:41:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names +18-08-2023 12:41:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:03 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:08 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:13 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:17 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:18 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:20 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:42:20 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: cleaner terminated +18-08-2023 12:42:20 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:42:20 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:42:20 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:20 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4529001433918421359","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":900,"average_commit_lag_file_count":1,"start_time":1692387708038,"end_time":1692387740450,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:42:20 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: service closed +18-08-2023 12:42:20 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 +18-08-2023 12:42:20 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 is:0, names:[] +18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 dropped +18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 dropped +18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 +18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:21 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8187446402040964935 +18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 +18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:21 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:42:21 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:42:21 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:42:21 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:42:21 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:21 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, existing:[] +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 - service started +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8187446402040964935, topicPartition: test-0 +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:42:21 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 12:42:21 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8187446402040964935. +18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 retrieved 0 file names +18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935. +18-08-2023 12:42:21 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 doesn't exist +18-08-2023 12:42:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 retrieved 0 file names +18-08-2023 12:42:22 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: cleaner started +18-08-2023 12:42:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8187446402040964935","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387741467} of type:kafka_pipe_start +18-08-2023 12:42:22 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz +18-08-2023 12:42:22 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 +18-08-2023 12:42:22 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935, filePath:TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz +18-08-2023 12:42:22 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 +18-08-2023 12:42:22 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935, filePath:TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz +18-08-2023 12:42:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz +18-08-2023 12:42:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz] +18-08-2023 12:42:22 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz] +18-08-2023 12:42:22 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0/insertFiles?requestId=aa386756-be1f-4ffe-af99-9577fb3972e6&showSkippedFiles=false +18-08-2023 12:42:23 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:23 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:42:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:26 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names +18-08-2023 12:42:31 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:36 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:42:41 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:42:46 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:42:51 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:42:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: cleaner terminated +18-08-2023 12:42:54 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:42:54 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:42:54 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:42:54 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:42:54 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8187446402040964935","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1198,"average_commit_lag_file_count":1,"start_time":1692387741466,"end_time":1692387774319,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:42:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: service closed +18-08-2023 12:42:54 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 +18-08-2023 12:42:54 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 is:0, names:[] +18-08-2023 12:42:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 dropped +18-08-2023 12:42:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 dropped +18-08-2023 12:42:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 +18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7966792751296594456 +18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 +18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:55 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:42:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:42:55 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:42:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:42:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:55 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, existing:[] +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 - service started +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7966792751296594456, topicPartition: test-0 +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7966792751296594456. +18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 retrieved 0 file names +18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456. +18-08-2023 12:42:56 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 doesn't exist +18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 retrieved 0 file names +18-08-2023 12:42:56 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: cleaner started +18-08-2023 12:42:56 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7966792751296594456","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387775534} of type:kafka_pipe_start +18-08-2023 12:42:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz +18-08-2023 12:42:56 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 12:42:56 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456, filePath:TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz +18-08-2023 12:42:56 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 +18-08-2023 12:42:56 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456, filePath:TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz +18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz +18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz] +18-08-2023 12:42:56 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz] +18-08-2023 12:42:56 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0/insertFiles?requestId=c1ead534-38e3-4ad3-9519-3d9537117027&showSkippedFiles=false +18-08-2023 12:42:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:42:57 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:57 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:42:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:42:57 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0/insertReport?requestId=3644d08b-51db-4538-b30e-9a19fb61fe0d +18-08-2023 12:42:58 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:58 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@53c25f1b} +18-08-2023 12:42:58 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 1 +18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] Purging loaded files for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, loadedFileCount:1, loadedFiles:[TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] +18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] deleted TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz from stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:42:58 pool-4-thread-1 INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] purge 1 files from stage: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 +18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":7136,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":59989,"average_ingestion_lag_file_count":1,"average_commit_lag":1381,"average_commit_lag_file_count":1,"start_time":1692387717114,"end_time":1692387778353,"is_pipe_closing":false} of type:kafka_pipe_usage +18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names +18-08-2023 12:43:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: cleaner terminated +18-08-2023 12:43:00 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:43:00 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:43:00 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:43:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":7136,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692387778353,"end_time":1692387780165,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:43:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: service closed +18-08-2023 12:43:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 +18-08-2023 12:43:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 is:0, names:[] +18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 dropped +18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 dropped +18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_1 dropped +18-08-2023 12:43:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:00 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:05 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:10 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:15 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 +18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_1 +18-08-2023 12:43:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:43:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:43:16 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8464995679227078074 +18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 +18-08-2023 12:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 +18-08-2023 12:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:17 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:43:17 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 12:43:17 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:43:17 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:43:17 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:43:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:17 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0, existing:[] +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 - service started +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8464995679227078074, topicPartition: test-0 +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8464995679227078074. +18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 retrieved 0 file names +18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074. +18-08-2023 12:43:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 doesn't exist +18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 +18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 retrieved 0 file names +18-08-2023 12:43:17 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0: cleaner started +18-08-2023 12:43:18 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8464995679227078074","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387797337} of type:kafka_pipe_start +18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 dropped +18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 dropped +18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_1 dropped +18-08-2023 12:43:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:20 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:25 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:28 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: cleaner terminated +18-08-2023 12:43:28 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:43:28 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:43:28 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:43:28 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:28 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7966792751296594456","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1124,"average_commit_lag_file_count":1,"start_time":1692387775533,"end_time":1692387808167,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:43:28 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: service closed +18-08-2023 12:43:28 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 +18-08-2023 12:43:28 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 is:0, names:[] +18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 dropped +18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 dropped +18-08-2023 12:43:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:43:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 +18-08-2023 12:43:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8912050872937575474 +18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:43:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 +18-08-2023 12:43:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:29 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:43:29 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:43:29 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:43:29 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:43:29 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:29 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, existing:[] +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 - service started +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8912050872937575474, topicPartition: test-0 +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:43:29 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 12:43:29 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8912050872937575474. +18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 retrieved 0 file names +18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474. +18-08-2023 12:43:29 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 doesn't exist +18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 retrieved 0 file names +18-08-2023 12:43:29 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: cleaner started +18-08-2023 12:43:29 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8912050872937575474","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387809031} of type:kafka_pipe_start +18-08-2023 12:43:29 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz +18-08-2023 12:43:29 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 12:43:30 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474, filePath:TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz +18-08-2023 12:43:30 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 +18-08-2023 12:43:30 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474, filePath:TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz +18-08-2023 12:43:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz +18-08-2023 12:43:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz] +18-08-2023 12:43:30 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz] +18-08-2023 12:43:30 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0/insertFiles?requestId=95cafb91-27c8-4758-918b-b7aaf0e3fc5e&showSkippedFiles=false +18-08-2023 12:43:31 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:43:31 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:43:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:43:34 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:39 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:49 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:54 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:43:59 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: cleaner terminated +18-08-2023 12:44:01 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:44:01 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:44:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:44:01 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:01 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8912050872937575474","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1112,"average_commit_lag_file_count":1,"start_time":1692387809029,"end_time":1692387841831,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:44:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: service closed +18-08-2023 12:44:01 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 +18-08-2023 12:44:01 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 is:0, names:[] +18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 dropped +18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 dropped +18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:02 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:44:02 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 12:44:02 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 12:44:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:02 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:44:02 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:44:02 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:44:02 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 12:44:02 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 12:44:02 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +18-08-2023 12:44:02 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 12:44:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 12:44:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 12:44:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:44:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1196559338745700642 doesn't exist +18-08-2023 12:44:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1196559338745700642. +18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1196559338745700642 +18-08-2023 12:44:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1196559338745700642 +18-08-2023 12:44:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1196559338745700642, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_0","table":"kafka_connector_test_table_1196559338745700642"} +18-08-2023 12:44:03 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1196559338745700642, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:03 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642 +18-08-2023 12:44:03 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:03 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset:null +18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 12:44:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1196559338745700642, topicPartition: test-0 +18-08-2023 12:44:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 12:44:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1054, currentDiffMillis=1061 +18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 +18-08-2023 12:44:03 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1777552b[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec] +18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, idx=0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. +18-08-2023 12:44:03 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +18-08-2023 12:44:03 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +18-08-2023 12:44:04 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1105: footer length = 755 +18-08-2023 12:44:04 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 755 => 243 2 0 0 +18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1113, encryptedCompressedSize=1120, bdecVersion=THREE +18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, size=1120 +18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, size=1120, timeInMillis=207 +18-08-2023 12:44:04 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, idx=0 +18-08-2023 12:44:04 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 12:44:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 12:44:04 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec","md5":"e623cdb8f87165f05bfc3c1676775646","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","chunk_start_offset":0,"chunk_length":1113,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"0e01e2385f97de443ef3fea72f1f1da0","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370747854,"first_insert_time_in_ms":1692387843349,"last_insert_time_in_ms":1692387843349,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387843805,"build_duration_ms":254,"upload_duration_ms":204},"bdec_version":3}],"role":"testrole_kafka","request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_2"} +18-08-2023 12:44:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 12:44:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:44:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 12:44:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 12:44:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:07 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 12:44:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 12:44:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:44:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 +18-08-2023 12:44:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 12:44:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:12 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1038 +18-08-2023 12:44:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 12:44:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +18-08-2023 12:44:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 +18-08-2023 12:44:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1046 +18-08-2023 12:44:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:17 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +18-08-2023 12:44:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 +18-08-2023 12:44:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 +18-08-2023 12:44:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:44:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 12:44:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:22 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:44:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:44:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 12:44:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 12:44:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:27 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 12:44:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:44:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:44:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:44:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 12:44:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:32 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:44:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:44:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 +18-08-2023 12:44:34 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:44:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset:0 +18-08-2023 12:44:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 12:44:34 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:44:34 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +18-08-2023 12:44:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 12:44:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=621, currentDiffMillis=621 +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=308, min=0, max=21, mean=0.2548306063738825, stddev=1.7579762641061176, p50=0.0, p75=0.0, p95=0.0, p98=4.0, p99=12.0, p999=21.0 +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4401573727090455, rate_unit=events/second +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.03800807271313, rate_unit=events/second +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=254.494, max=254.494, mean=254.494, stddev=0.0, p50=254.494, p75=254.494, p95=254.494, p98=254.494, p99=254.494, p999=254.494, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03128272466275268, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=621.549875, max=621.549875, mean=621.549875, stddev=0.0, p50=621.549875, p75=621.549875, p95=621.549875, p98=621.549875, p99=621.549875, p999=621.549875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282568575417946, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=157.005708, max=157.005708, mean=157.005708, stddev=0.0, p50=157.005708, p75=157.005708, p95=157.005708, p98=157.005708, p99=157.005708, p999=157.005708, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282468106885765, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=204.639709, max=204.639709, mean=204.639709, stddev=0.0, p50=204.639709, p75=204.639709, p95=204.639709, p98=204.639709, p99=204.639709, p999=204.639709, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282337913482866, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:44:34 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:34 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:44:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 12:44:34 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:44:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:44:35 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 12:44:35 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 12:44:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 12:44:35 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:35 Thread-18 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:44:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:44:35 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:44:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 12:44:35 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 12:44:35 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 12:44:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:44:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_9105461817253957002 doesn't exist +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_9105461817253957002. +18-08-2023 12:44:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9105461817253957002 +18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_9105461817253957002 +18-08-2023 12:44:35 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_9105461817253957002, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_0","table":"kafka_connector_test_table_9105461817253957002"} +18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_9105461817253957002, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002 +18-08-2023 12:44:35 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset:null +18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_9105461817253957002, topicPartition: test-0 +18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:44:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 12:44:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} +18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 +18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 12:44:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:44:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 +18-08-2023 12:44:36 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@4e0ddc23[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, idx=0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1155: footer length = 775 +18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 775 => 7 3 0 0 +18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1163, encryptedCompressedSize=1168, bdecVersion=THREE +18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, size=1168 +18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, size=1168, timeInMillis=287 +18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, idx=0 +18-08-2023 12:44:36 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 12:44:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 12:44:36 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec","md5":"126aae5f4fb8c18cdb76f15cfec1887a","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","chunk_start_offset":0,"chunk_length":1163,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"2ac552b25f8368a0281516c1da166a7e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370745286,"first_insert_time_in_ms":1692387875754,"last_insert_time_in_ms":1692387875754,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387876171,"build_duration_ms":9,"upload_duration_ms":286},"bdec_version":3}],"role":"testrole_kafka","request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_2"} +18-08-2023 12:44:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 12:44:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +18-08-2023 12:44:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 +18-08-2023 12:44:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 12:44:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:40 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1022, currentDiffMillis=1023 +18-08-2023 12:44:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:44:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:44:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 12:44:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 12:44:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:45 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 +18-08-2023 12:44:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 12:44:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 +18-08-2023 12:44:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1052, currentDiffMillis=1053 +18-08-2023 12:44:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 12:44:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:50 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:44:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 +18-08-2023 12:44:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:44:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +18-08-2023 12:44:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 12:44:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:55 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:44:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +18-08-2023 12:44:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:44:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 12:44:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:44:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 12:44:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:45:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:00 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 12:45:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 12:45:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:45:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 12:45:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:05 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:45:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 12:45:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:06 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 +18-08-2023 12:45:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:06 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset:1 +18-08-2023 12:45:06 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 12:45:06 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:06 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 12:45:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 12:45:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=493, currentDiffMillis=493 +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=9, mean=0.09705703854644274, stddev=0.7988878636542661, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=4.0, p999=9.0 +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.6156135321258924, rate_unit=events/second +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.80760150999839, rate_unit=events/second +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=9.518958, max=9.518958, mean=9.518958, stddev=0.0, p50=9.518958, p75=9.518958, p95=9.518958, p98=9.518958, p99=9.518958, p999=9.518958, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0315132618714113, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=434.757167, max=434.757167, mean=434.757167, stddev=0.0, p50=434.757167, p75=434.757167, p95=434.757167, p98=434.757167, p99=434.757167, p999=434.757167, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03151318184579166, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=137.858209, max=137.858209, mean=137.858209, stddev=0.0, p50=137.858209, p75=137.858209, p95=137.858209, p98=137.858209, p99=137.858209, p999=137.858209, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031513128674358286, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=286.275125, max=286.275125, mean=286.275125, stddev=0.0, p50=286.275125, p75=286.275125, p95=286.275125, p98=286.275125, p99=286.275125, p999=286.275125, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03151304310509383, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:06 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:45:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:45:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 12:45:06 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:45:07 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 12:45:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:07 Thread-20 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:45:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:45:07 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:45:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 12:45:07 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 12:45:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 +18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 12:45:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:45:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_7874228197508598970 doesn't exist +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_7874228197508598970. +18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7874228197508598970 +18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_7874228197508598970 +18-08-2023 12:45:07 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7874228197508598970, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_0","table":"kafka_connector_test_table_7874228197508598970"} +18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7874228197508598970, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970 +18-08-2023 12:45:07 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset:null +18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7874228197508598970, topicPartition: test-0 +18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1024 +18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 +18-08-2023 12:45:08 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@59f2a0f0[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, idx=0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1102: footer length = 752 +18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 752 => 240 2 0 0 +18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1110, encryptedCompressedSize=1120, bdecVersion=THREE +18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, size=1120 +18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, size=1120, timeInMillis=226 +18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, idx=0 +18-08-2023 12:45:08 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 12:45:08 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +18-08-2023 12:45:08 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec","md5":"112aaf108f3d79012aca70081d3f4e27","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","chunk_start_offset":0,"chunk_length":1110,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"1fcb88c3acf901bbb3afd81475cc720e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370745514,"first_insert_time_in_ms":1692387907912,"last_insert_time_in_ms":1692387907912,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387908331,"build_duration_ms":4,"upload_duration_ms":225},"bdec_version":3}],"role":"testrole_kafka","request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_2"} +18-08-2023 12:45:08 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +18-08-2023 12:45:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:45:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_1 +18-08-2023 12:45:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:45:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 12:45:10 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 12:45:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:45:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8383920415137154704 +18-08-2023 12:45:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 +18-08-2023 12:45:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:45:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:11 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:45:11 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 12:45:11 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:45:11 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:45:11 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:45:11 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:11 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, existing:[] +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 - service started +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8383920415137154704, topicPartition: test-0 +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +18-08-2023 12:45:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8383920415137154704. +18-08-2023 12:45:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names +18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704. +18-08-2023 12:45:11 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 doesn't exist +18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names +18-08-2023 12:45:12 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: cleaner started +18-08-2023 12:45:12 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387911341} of type:kafka_pipe_start +18-08-2023 12:45:12 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz +18-08-2023 12:45:12 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 3 records, 5492 bytes, offset 0 - 2 +18-08-2023 12:45:12 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:12 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704, filePath:TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz +18-08-2023 12:45:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 12:45:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:12 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:45:12 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704, filePath:TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz +18-08-2023 12:45:12 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz +18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names +18-08-2023 12:45:12 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] +18-08-2023 12:45:12 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] +18-08-2023 12:45:12 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0/insertFiles?requestId=9dd4f6c2-d041-429e-ad4f-0f1d22828e66&showSkippedFiles=false +18-08-2023 12:45:13 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:45:13 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:45:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 12:45:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names +18-08-2023 12:45:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 12:45:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +18-08-2023 12:45:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +18-08-2023 12:45:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:17 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 12:45:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 12:45:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:45:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 12:45:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 12:45:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:22 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 12:45:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 12:45:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 12:45:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 12:45:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1046 +18-08-2023 12:45:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:27 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +18-08-2023 12:45:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 12:45:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +18-08-2023 12:45:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 12:45:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 12:45:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:32 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 12:45:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 12:45:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 12:45:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +18-08-2023 12:45:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:37 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1022, currentDiffMillis=1022 +18-08-2023 12:45:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 12:45:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 +18-08-2023 12:45:38 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset:0 +18-08-2023 12:45:38 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 12:45:38 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:38 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +18-08-2023 12:45:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 12:45:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=367, currentDiffMillis=367 +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=305, min=0, max=2, mean=0.01293826210198239, stddev=0.13413729964389748, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=1.0, p999=2.0 +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4735974066205038, rate_unit=events/second +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.51821928265969, rate_unit=events/second +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=4.477, max=4.477, mean=4.477, stddev=0.0, p50=4.477, p75=4.477, p95=4.477, p98=4.477, p99=4.477, p999=4.477, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031712433596499674, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=370.255416, max=370.255416, mean=370.255416, stddev=0.0, p50=370.255416, p75=370.255416, p95=370.255416, p98=370.255416, p99=370.255416, p999=370.255416, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031712061626553704, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=139.045125, max=139.045125, mean=139.045125, stddev=0.0, p50=139.045125, p75=139.045125, p95=139.045125, p98=139.045125, p99=139.045125, p999=139.045125, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03171158273189406, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=225.996042, max=225.996042, mean=225.996042, stddev=0.0, p50=225.996042, p75=225.996042, p95=225.996042, p98=225.996042, p99=225.996042, p999=225.996042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031711420784943704, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:45:38 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:38 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:45:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 12:45:38 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:45:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 12:45:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 12:45:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:39 Thread-22 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:45:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:45:39 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:45:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 12:45:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 12:45:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 +18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 12:45:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:45:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 12:45:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 12:45:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:45:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4720053319416705063 doesn't exist +18-08-2023 12:45:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4720053319416705063. +18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4720053319416705063 +18-08-2023 12:45:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4720053319416705063 +18-08-2023 12:45:39 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4720053319416705063, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_0","table":"kafka_connector_test_table_4720053319416705063"} +18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4720053319416705063, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063 +18-08-2023 12:45:39 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 12:45:39 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:45:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset:null +18-08-2023 12:45:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 12:45:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4720053319416705063, topicPartition: test-0 +18-08-2023 12:45:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:45:40 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 12:45:40 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} +18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 +18-08-2023 12:45:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 12:45:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 +18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 +18-08-2023 12:45:40 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@28b462e9[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec] +18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, idx=0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1113: footer length = 751 +18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 751 => 239 2 0 0 +18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1121, encryptedCompressedSize=1136, bdecVersion=THREE +18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, size=1136 +18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, size=1136, timeInMillis=199 +18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, idx=0 +18-08-2023 12:45:40 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 12:45:40 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +18-08-2023 12:45:40 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec","md5":"2ccbb04334296489837e3cab3e54dc3c","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","chunk_start_offset":0,"chunk_length":1121,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"f5cf44207772ff02ae74e8f7c8a0ee82","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370748686,"first_insert_time_in_ms":1692387940000,"last_insert_time_in_ms":1692387940000,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387940348,"build_duration_ms":8,"upload_duration_ms":198},"bdec_version":3}],"role":"testrole_kafka","request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_2"} +18-08-2023 12:45:40 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +18-08-2023 12:45:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 +18-08-2023 12:45:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 12:45:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1052 +18-08-2023 12:45:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:45:44 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names +18-08-2023 12:45:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 12:45:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:45:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +18-08-2023 12:45:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 12:45:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 12:45:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:49 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 12:45:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 12:45:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:45:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 12:45:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 12:45:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 12:45:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:54 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:45:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 12:45:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 +18-08-2023 12:45:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:45:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 12:45:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 12:45:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 +18-08-2023 12:45:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:45:59 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:46:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 12:46:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 12:46:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:46:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 12:46:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 +18-08-2023 12:46:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:04 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:46:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 12:46:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 12:46:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:46:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 12:46:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 12:46:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 12:46:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:09 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:46:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 12:46:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:46:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +18-08-2023 12:46:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:10 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 +18-08-2023 12:46:10 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:46:10 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset:1 +18-08-2023 12:46:10 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 12:46:10 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 12:46:10 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 12:46:11 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 12:46:11 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 12:46:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=527, currentDiffMillis=527 +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=8, mean=0.07917226020632413, stddev=0.6470555470389748, p50=0.0, p75=0.0, p95=0.0, p98=1.0, p99=4.0, p999=8.0 +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.422444063774894, rate_unit=events/second +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.738717882934, rate_unit=events/second +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=8.852083, max=8.852083, mean=8.852083, stddev=0.0, p50=8.852083, p75=8.852083, p95=8.852083, p98=8.852083, p99=8.852083, p999=8.852083, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03145986490791663, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=371.17925, max=371.17925, mean=371.17925, stddev=0.0, p50=371.17925, p75=371.17925, p95=371.17925, p98=371.17925, p99=371.17925, p999=371.17925, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0314596827177743, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=162.694458, max=162.694458, mean=162.694458, stddev=0.0, p50=162.694458, p75=162.694458, p95=162.694458, p98=162.694458, p99=162.694458, p999=162.694458, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031459540653228084, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=198.545875, max=198.545875, mean=198.545875, stddev=0.0, p50=198.545875, p75=198.545875, p95=198.545875, p98=198.545875, p99=198.545875, p999=198.545875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03145935793141843, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 12:46:11 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:46:11 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:46:11 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 12:46:11 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:46:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:46:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:46:12 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0/insertReport?requestId=2019ea59-9641-49e5-99d5-36b56c767dcb +18-08-2023 12:46:12 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:46:12 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@75b2ff9} +18-08-2023 12:46:12 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 1 +18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] Purging loaded files for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, loadedFileCount:1, loadedFiles:[TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] +18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] deleted TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz from stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:46:12 pool-4-thread-1 INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] purge 1 files from stage: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 +18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":5492,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":59991,"average_ingestion_lag_file_count":1,"average_commit_lag":1356,"average_commit_lag_file_count":1,"start_time":1692387911337,"end_time":1692387972798,"is_pipe_closing":false} of type:kafka_pipe_usage +18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names +18-08-2023 12:46:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: cleaner terminated +18-08-2023 12:46:14 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:46:14 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:46:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:46:14 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:46:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":5492,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692387972798,"end_time":1692387974518,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:46:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: service closed +18-08-2023 12:46:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 +18-08-2023 12:46:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 is:0, names:[] +18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 dropped +18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 dropped +18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_1 dropped +18-08-2023 12:46:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3264218329753027732 +18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3264218329753027732_0 +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3264218329753027732 dropped +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3264218329753027732_0 dropped +18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9072147538436586285 +18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9072147538436586285_0 +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9072147538436586285 dropped +18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9072147538436586285_0 dropped +18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3050797644145311050 +18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3050797644145311050_0 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3050797644145311050 dropped +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3050797644145311050_0 dropped +18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2363792425311133864 +18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2363792425311133864_0 +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2363792425311133864 dropped +18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2363792425311133864_0 dropped +18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:48:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:48:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:48:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:48:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 +18-08-2023 12:48:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8546433757402467292 +18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:48:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 +18-08-2023 12:48:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:30 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:48:30 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 12:48:30 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:48:30 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:48:30 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:48:30 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:30 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, existing:[] +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 - service started +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8546433757402467292, topicPartition: test-0 +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8546433757402467292. +18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 retrieved 0 file names +18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292. +18-08-2023 12:48:30 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 doesn't exist +18-08-2023 12:48:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:48:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 retrieved 0 file names +18-08-2023 12:48:31 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: cleaner started +18-08-2023 12:48:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8546433757402467292","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388110425} of type:kafka_pipe_start +18-08-2023 12:48:31 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz +18-08-2023 12:48:31 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 12:48:31 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292, filePath:TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz +18-08-2023 12:48:31 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 +18-08-2023 12:48:31 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292, filePath:TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz +18-08-2023 12:48:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz +18-08-2023 12:48:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz] +18-08-2023 12:48:31 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz] +18-08-2023 12:48:31 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0/insertFiles?requestId=6ff1ca31-015a-4a4d-889b-910b09eee40c&showSkippedFiles=false +18-08-2023 12:48:32 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:48:32 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 12:48:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:48:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:48:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:48:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:48:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:48:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:49:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:49:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: cleaner terminated +18-08-2023 12:49:03 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 12:49:03 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 12:49:03 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 12:49:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 12:49:03 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8546433757402467292","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1278,"average_commit_lag_file_count":1,"start_time":1692388110417,"end_time":1692388143128,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 12:49:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: service closed +18-08-2023 12:49:03 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 +18-08-2023 12:49:03 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 is:0, names:[] +18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 dropped +18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 dropped +18-08-2023 12:49:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 12:49:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 12:49:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:04 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 12:49:04 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 12:49:04 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 12:49:04 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 12:49:04 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:04 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0, existing:[] +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 - service started +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_515941071091186449, topicPartition: test-0 +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 12:49:04 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 12:49:04 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 12:49:04 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_515941071091186449. +18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 retrieved 0 file names +18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449. +18-08-2023 12:49:04 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 doesn't exist +18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 +18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 retrieved 0 file names +18-08-2023 12:49:04 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0: cleaner started +18-08-2023 12:49:04 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388144104} of type:kafka_pipe_start +18-08-2023 12:49:04 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_key_1692388144825.gz +18-08-2023 12:49:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_key_1692388144825.gz to table stage: kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:05 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_value_1692388145170.gz +18-08-2023 12:49:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_value_1692388145170.gz to table stage: kafka_connector_test_table_515941071091186449 +18-08-2023 12:49:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:49:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:49:59 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:04 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:04 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692388144015,"end_time":1692388204837,"is_pipe_closing":false} of type:kafka_pipe_usage +18-08-2023 12:50:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:50:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:50:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:50:59 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:51:04 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 12:51:04 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692388204837,"end_time":1692388264835,"is_pipe_closing":false} of type:kafka_pipe_usage +18-08-2023 12:51:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 12:51:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:00:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 +18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4136623387971130024 +18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 +18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:24 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:00:24 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:00:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:00:24 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:00:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:00:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, existing:[] +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 - service started +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4136623387971130024, topicPartition: test-0 +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4136623387971130024. +18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 retrieved 0 file names +18-08-2023 13:00:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024. +18-08-2023 13:00:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 doesn't exist +18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 retrieved 0 file names +18-08-2023 13:00:25 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: cleaner started +18-08-2023 13:00:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4136623387971130024","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388824753} of type:kafka_pipe_start +18-08-2023 13:00:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz +18-08-2023 13:00:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:00:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024, filePath:TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz +18-08-2023 13:00:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 +18-08-2023 13:00:26 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024, filePath:TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz +18-08-2023 13:00:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz +18-08-2023 13:00:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz] +18-08-2023 13:00:26 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz] +18-08-2023 13:00:26 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0/insertFiles?requestId=2c223d0a-7b31-49c4-9c07-73804a59d4c2&showSkippedFiles=false +18-08-2023 13:00:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:00:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:00:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:29 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:34 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:39 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:49 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:54 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:00:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: cleaner terminated +18-08-2023 13:00:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:00:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:00:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:00:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4136623387971130024","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1254,"average_commit_lag_file_count":1,"start_time":1692388824745,"end_time":1692388857570,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:00:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: service closed +18-08-2023 13:00:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 +18-08-2023 13:00:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 is:0, names:[] +18-08-2023 13:00:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 dropped +18-08-2023 13:00:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 dropped +18-08-2023 13:00:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 +18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7116658905857989900 +18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 +18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:58 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:00:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:00:58 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:00:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:00:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0, existing:[] +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 - service started +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7116658905857989900, topicPartition: test-0 +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:00:58 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 13:00:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:00:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7116658905857989900. +18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 retrieved 0 file names +18-08-2023 13:00:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900. +18-08-2023 13:00:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 doesn't exist +18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 +18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 retrieved 0 file names +18-08-2023 13:00:59 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0: cleaner started +18-08-2023 13:00:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7116658905857989900","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388858714} of type:kafka_pipe_start +18-08-2023 13:00:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_key_1692388859543.gz +18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_key_1692388859543.gz to table stage: kafka_connector_test_table_7116658905857989900 +18-08-2023 13:00:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_value_1692388859956.gz +18-08-2023 13:01:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_value_1692388859956.gz to table stage: kafka_connector_test_table_7116658905857989900 +18-08-2023 13:01:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:01:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:01:33 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:01:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:01:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 +18-08-2023 13:01:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4080866019933655485 +18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:01:50 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 +18-08-2023 13:01:50 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:50 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:01:50 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:01:50 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:01:50 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:01:50 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:01:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:50 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, existing:[] +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 - service started +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4080866019933655485, topicPartition: test-0 +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4080866019933655485. +18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 retrieved 0 file names +18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485. +18-08-2023 13:01:50 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 doesn't exist +18-08-2023 13:01:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:01:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 retrieved 0 file names +18-08-2023 13:01:51 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: cleaner started +18-08-2023 13:01:51 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4080866019933655485","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388910373} of type:kafka_pipe_start +18-08-2023 13:01:51 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz +18-08-2023 13:01:51 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:01:51 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485, filePath:TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz +18-08-2023 13:01:51 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 +18-08-2023 13:01:51 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485, filePath:TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz +18-08-2023 13:01:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz +18-08-2023 13:01:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz] +18-08-2023 13:01:51 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz] +18-08-2023 13:01:51 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0/insertFiles?requestId=397783b8-a935-4025-964d-58f6bc46aff0&showSkippedFiles=false +18-08-2023 13:01:52 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:01:52 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:01:52 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:01:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:23 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:02:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: cleaner terminated +18-08-2023 13:02:23 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:02:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:02:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:02:23 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4080866019933655485","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1266,"average_commit_lag_file_count":1,"start_time":1692388910365,"end_time":1692388943363,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:02:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: service closed +18-08-2023 13:02:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 +18-08-2023 13:02:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 is:0, names:[] +18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 dropped +18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 dropped +18-08-2023 13:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:02:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:02:24 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:02:24 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:02:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:02:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0, existing:[] +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 - service started +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_487147896604185713, topicPartition: test-0 +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:02:24 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 13:02:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:02:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_487147896604185713. +18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 retrieved 0 file names +18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713. +18-08-2023 13:02:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 doesn't exist +18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 +18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 retrieved 0 file names +18-08-2023 13:02:25 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0: cleaner started +18-08-2023 13:02:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_487147896604185713","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388944320} of type:kafka_pipe_start +18-08-2023 13:02:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_key_1692388945210.gz +18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_key_1692388945210.gz to table stage: kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_value_1692388945600.gz +18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_value_1692388945600.gz to table stage: kafka_connector_test_table_487147896604185713 +18-08-2023 13:02:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:54 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:02:54 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:02:54 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:02:54 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:02:54 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:02:54 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:54 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0, existing:[] +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 - service started +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5916591702771815345, topicPartition: test-0 +18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:02:55 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 13:02:55 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:02:55 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5916591702771815345. +18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 retrieved 0 file names +18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345. +18-08-2023 13:02:55 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 doesn't exist +18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 +18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 retrieved 0 file names +18-08-2023 13:02:55 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0: cleaner started +18-08-2023 13:02:55 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5916591702771815345","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388975041} of type:kafka_pipe_start +18-08-2023 13:02:55 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_key_1692388975848.gz +18-08-2023 13:02:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:02:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_key_1692388975848.gz to table stage: kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_value_1692388976385.gz +18-08-2023 13:02:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_value_1692388976385.gz to table stage: kafka_connector_test_table_5916591702771815345 +18-08-2023 13:02:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:03:02 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:02 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:37 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:03:37 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:03:37 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:03:37 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:03:37 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:03:37 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:37 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0, existing:[] +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 - service started +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4425459254925033856, topicPartition: test-0 +18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:03:38 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 13:03:38 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:03:38 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:03:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:03:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4425459254925033856. +18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 retrieved 0 file names +18-08-2023 13:03:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856. +18-08-2023 13:03:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 doesn't exist +18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 +18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 retrieved 0 file names +18-08-2023 13:03:43 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0: cleaner started +18-08-2023 13:03:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4425459254925033856","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692389022721} of type:kafka_pipe_start +18-08-2023 13:03:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_key_1692389023506.gz +18-08-2023 13:03:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_key_1692389023506.gz to table stage: kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:44 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_value_1692389024041.gz +18-08-2023 13:03:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_value_1692389024041.gz to table stage: kafka_connector_test_table_4425459254925033856 +18-08-2023 13:03:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:03:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:03:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:03:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:07 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:12 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:17 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:04:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:04:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:04:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:04:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:04:52 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:52 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:04:53 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:53 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:53 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:04:53 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:04:53 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:04:53 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:04:53 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:04:53 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:53 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0, existing:[] +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 - service started +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5251568966378274834, topicPartition: test-0 +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:04:53 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 13:04:53 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:04:53 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5251568966378274834. +18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 retrieved 0 file names +18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834. +18-08-2023 13:04:53 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 doesn't exist +18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 +18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 retrieved 0 file names +18-08-2023 13:04:53 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0: cleaner started +18-08-2023 13:04:53 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5251568966378274834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692389093264} of type:kafka_pipe_start +18-08-2023 13:04:53 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_key_1692389093942.gz +18-08-2023 13:04:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_key_1692389093942.gz to table stage: kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:54 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_value_1692389094485.gz +18-08-2023 13:04:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_value_1692389094485.gz to table stage: kafka_connector_test_table_5251568966378274834 +18-08-2023 13:04:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:04:58 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:13 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:18 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:05:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:05:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:06:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:07:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:07:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:07:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:07:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:07:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 +18-08-2023 13:07:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 +18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_304205135411409227 +18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:07:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 +18-08-2023 13:07:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 +18-08-2023 13:07:04 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:07:04 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:07:04 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:07:04 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:07:04 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:07:04 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 +18-08-2023 13:07:04 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0, existing:[] +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 - service started +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_304205135411409227, topicPartition: test-0 +18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 dropped +18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 dropped +18-08-2023 13:07:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:08:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:08:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:08:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:08:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 +18-08-2023 13:08:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 +18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5819610730455116141 +18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:08:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 +18-08-2023 13:08:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 +18-08-2023 13:08:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:08:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:08:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:08:32 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:08:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:08:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 +18-08-2023 13:08:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0, existing:[] +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 - service started +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5819610730455116141, topicPartition: test-0 +18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 dropped +18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 dropped +18-08-2023 13:08:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:10:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:10:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 +18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 +18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7486010370272619727 +18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 +18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 +18-08-2023 13:10:56 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:10:56 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:10:56 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:10:56 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:10:56 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:10:56 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 +18-08-2023 13:10:56 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0, existing:[] +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 - service started +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7486010370272619727, topicPartition: test-0 +18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:10:56 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:10:56 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:10:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:10:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0: cleaner terminated +18-08-2023 13:10:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:10:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:10:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0: service closed +18-08-2023 13:10:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 13:10:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 +18-08-2023 13:10:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 is:0, names:[] +18-08-2023 13:10:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 dropped +18-08-2023 13:10:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 dropped +18-08-2023 13:10:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:17:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:17:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 +18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 +18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3983027369971842529 +18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 +18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 +18-08-2023 13:17:08 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:17:08 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:17:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:17:08 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:17:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:17:08 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 +18-08-2023 13:17:08 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0, existing:[] +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 - service started +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3983027369971842529, topicPartition: test-0 +18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 dropped +18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 dropped +18-08-2023 13:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:21:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2327466389014140672 +18-08-2023 13:21:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2327466389014140672_0 +18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2327466389014140672 dropped +18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2327466389014140672_0 dropped +18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:27 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:21:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 +18-08-2023 13:21:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 +18-08-2023 13:21:37 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:37 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. +18-08-2023 13:21:37 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:37 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:37 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1932307370833503227 +18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:21:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 +18-08-2023 13:21:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 +18-08-2023 13:21:38 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:21:38 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:21:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:21:38 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:21:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:21:38 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 +18-08-2023 13:21:38 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0, existing:[] +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 - service started +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1932307370833503227, topicPartition: test-0 +18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:21:38 main DEBUG RestService:260 - Sending POST with input {"schema":"{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"int16\",\"type\":{\"type\":\"int\",\"connect.type\":\"int16\"}}]}"} to http://fake-url/subjects/test-value/versions?normalize=false +18-08-2023 13:21:38 main ERROR RestService:276 - Failed to send HTTP request to endpoint: http://fake-url/subjects/test-value/versions?normalize=false +java.net.UnknownHostException: fake-url + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:229) + at java.base/java.net.Socket.connect(Socket.java:609) + at java.base/sun.net.NetworkClient.doConnect(NetworkClient.java:177) + at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:508) + at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:603) + at java.base/sun.net.www.http.HttpClient.(HttpClient.java:276) + at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:375) + at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:396) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1253) + at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1187) + at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:1081) + at java.base/sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:1015) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream0(HttpURLConnection.java:1367) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream(HttpURLConnection.java:1342) + at io.confluent.kafka.schemaregistry.client.rest.RestService.sendHttpRequest(RestService.java:272) + at io.confluent.kafka.schemaregistry.client.rest.RestService.httpRequest(RestService.java:371) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:548) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:536) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:494) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.registerAndGetId(CachedSchemaRegistryClient.java:274) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:381) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:354) + at io.confluent.kafka.serializers.AbstractKafkaAvroSerializer.serializeImpl(AbstractKafkaAvroSerializer.java:125) + at io.confluent.connect.avro.AvroConverter$Serializer.serialize(AvroConverter.java:153) + at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:86) + at com.snowflake.kafka.connector.internal.TombstoneRecordIngestionIT.testIgnoreTombstoneRecordBehavior(TombstoneRecordIngestionIT.java:208) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 dropped +18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 dropped +18-08-2023 13:21:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:21:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:21:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:21:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:21:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:21:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:21:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:21:39 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:21:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:21:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:21:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:21:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +18-08-2023 13:21:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:21:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:21:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:21:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:21:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:21:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6031348541095650961 doesn't exist +18-08-2023 13:21:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6031348541095650961. +18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6031348541095650961 +18-08-2023 13:21:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_6031348541095650961 +18-08-2023 13:21:39 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6031348541095650961, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:21:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"wKkrlehYLnc2nFjUQjRSPOP5YQ7uuMMajhbOAAYHOIgcCC_1003_0","table":"kafka_connector_test_table_6031348541095650961"} +18-08-2023 13:21:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6031348541095650961, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:21:40 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961 +18-08-2023 13:21:40 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:21:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0 +18-08-2023 13:21:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wKkrlehYLnc2nFjUQjRSPOP5YQ7uuMMajhbOAAYHOIgcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:21:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, offset:null +18-08-2023 13:21:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:21:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6031348541095650961, topicPartition: test-0 +18-08-2023 13:21:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:21:40 main DEBUG RestService:260 - Sending POST with input {"schema":"{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"int16\",\"type\":{\"type\":\"int\",\"connect.type\":\"int16\"}}]}"} to http://fake-url/subjects/test-value/versions?normalize=false +18-08-2023 13:21:40 main ERROR RestService:276 - Failed to send HTTP request to endpoint: http://fake-url/subjects/test-value/versions?normalize=false +java.net.UnknownHostException: fake-url + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:229) + at java.base/java.net.Socket.connect(Socket.java:609) + at java.base/sun.net.NetworkClient.doConnect(NetworkClient.java:177) + at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:508) + at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:603) + at java.base/sun.net.www.http.HttpClient.(HttpClient.java:276) + at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:375) + at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:396) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1253) + at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1187) + at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:1081) + at java.base/sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:1015) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream0(HttpURLConnection.java:1367) + at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream(HttpURLConnection.java:1342) + at io.confluent.kafka.schemaregistry.client.rest.RestService.sendHttpRequest(RestService.java:272) + at io.confluent.kafka.schemaregistry.client.rest.RestService.httpRequest(RestService.java:371) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:548) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:536) + at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:494) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.registerAndGetId(CachedSchemaRegistryClient.java:274) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:381) + at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:354) + at io.confluent.kafka.serializers.AbstractKafkaAvroSerializer.serializeImpl(AbstractKafkaAvroSerializer.java:125) + at io.confluent.connect.avro.AvroConverter$Serializer.serialize(AvroConverter.java:153) + at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:86) + at com.snowflake.kafka.connector.internal.TombstoneRecordIngestionIT.testIgnoreTombstoneRecordBehavior(TombstoneRecordIngestionIT.java:208) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +18-08-2023 13:21:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:25:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 +18-08-2023 13:25:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:13 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:25:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7700526656325775771 +18-08-2023 13:25:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:25:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 +18-08-2023 13:25:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:14 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:25:14 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:25:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:25:14 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:25:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:25:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:14 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, existing:[] +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 - service started +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7700526656325775771, topicPartition: test-0 +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:25:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:25:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7700526656325775771. +18-08-2023 13:25:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 retrieved 0 file names +18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771. +18-08-2023 13:25:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 doesn't exist +18-08-2023 13:25:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 retrieved 0 file names +18-08-2023 13:25:15 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: cleaner started +18-08-2023 13:25:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7700526656325775771","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390314362} of type:kafka_pipe_start +18-08-2023 13:25:15 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz +18-08-2023 13:25:15 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 +18-08-2023 13:25:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771, filePath:TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz +18-08-2023 13:25:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 +18-08-2023 13:25:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771, filePath:TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz +18-08-2023 13:25:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz +18-08-2023 13:25:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz] +18-08-2023 13:25:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz] +18-08-2023 13:25:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0/insertFiles?requestId=648c4ee4-096c-4e1a-bd5f-a3e13508a1be&showSkippedFiles=false +18-08-2023 13:25:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:25:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:25:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:19 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:24 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:29 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:34 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:39 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:47 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:25:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: cleaner terminated +18-08-2023 13:25:47 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:25:47 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:25:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:47 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7700526656325775771","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1306,"average_commit_lag_file_count":1,"start_time":1692390314321,"end_time":1692390347574,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:25:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: service closed +18-08-2023 13:25:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 +18-08-2023 13:25:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 is:0, names:[] +18-08-2023 13:25:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 dropped +18-08-2023 13:25:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 dropped +18-08-2023 13:25:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:25:48 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:48 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:48 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:25:48 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:25:48 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:25:48 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:25:48 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:48 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:25:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:25:48 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:25:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:25:48 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:25:48 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:25:48 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +18-08-2023 13:25:48 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:25:48 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:25:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:25:48 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:25:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:25:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4977398750416568100 doesn't exist +18-08-2023 13:25:48 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4977398750416568100. +18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4977398750416568100 +18-08-2023 13:25:48 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4977398750416568100 +18-08-2023 13:25:48 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4977398750416568100, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:25:48 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_0","table":"kafka_connector_test_table_4977398750416568100"} +18-08-2023 13:25:49 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4977398750416568100, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:25:49 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100 +18-08-2023 13:25:49 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:49 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset:null +18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:25:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4977398750416568100, topicPartition: test-0 +18-08-2023 13:25:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:25:49 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:25:49 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0,currentBufferSizeInBytes:211, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1} +18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 211 bytes, offset 1 - 1 +18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 13:25:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:25:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1038 +18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 +18-08-2023 13:25:49 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1a54a74d[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec] +18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, idx=0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. +18-08-2023 13:25:49 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 105, 110, 116, 49, 54, 34, 58, 49, 50, 125] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(12 bytes) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 16 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 16 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 40 to byteArray of 40 bytes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 40 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 63 to byteArray of 63 bytes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 168: end column +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 63B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 22B raw, 40B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 168: end block +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 168: column indexes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 330: offset indexes +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 352: bloom filters +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 352: end +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1088: footer length = 736 +18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 736 => 224 2 0 0 +18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100, rowCount=1, startOffset=0, estimatedUncompressedSize=74.5, paddedChunkLength=1096, encryptedCompressedSize=1104, bdecVersion=THREE +18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, size=1104 +18-08-2023 13:25:50 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, size=1104, timeInMillis=226 +18-08-2023 13:25:50 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, idx=0 +18-08-2023 13:25:50 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 13:25:50 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 13:25:50 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec","md5":"5e965e0219d5aee662bf744dca2310b8","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","chunk_start_offset":0,"chunk_length":1096,"chunk_length_uncompressed":74,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"5c1f25772925ad600e144c2f604d1e6c","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370812214,"first_insert_time_in_ms":1692390349190,"last_insert_time_in_ms":1692390349190,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390349512,"build_duration_ms":269,"upload_duration_ms":223},"bdec_version":3}],"role":"testrole_kafka","request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_2"} +18-08-2023 13:25:50 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 13:25:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 +18-08-2023 13:25:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +18-08-2023 13:25:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 13:25:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:53 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 +18-08-2023 13:25:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1043 +18-08-2023 13:25:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:25:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:25:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +18-08-2023 13:25:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:58 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:25:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1051 +18-08-2023 13:25:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:25:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 13:25:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1043 +18-08-2023 13:26:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 13:26:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 13:26:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:26:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 +18-08-2023 13:26:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:26:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 13:26:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:26:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:26:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:26:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 +18-08-2023 13:26:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 +18-08-2023 13:26:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:26:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:26:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:26:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 +18-08-2023 13:26:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1028 +18-08-2023 13:26:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 13:26:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 +18-08-2023 13:26:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +18-08-2023 13:26:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:26:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:26:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:26:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:26:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 +18-08-2023 13:26:20 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:26:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset:1 +18-08-2023 13:26:20 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 13:26:20 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:26:20 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 13:26:20 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 13:26:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=545, currentDiffMillis=545 +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1104, max=1104, mean=1104.0, stddev=0.0, p50=1104.0, p75=1104.0, p95=1104.0, p98=1104.0, p99=1104.0, p999=1104.0 +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=18, mean=0.3540181330563173, stddev=1.9300136683346523, p50=0.0, p75=0.0, p95=1.0, p98=8.0, p99=11.0, p999=18.0 +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=74, m1_rate=9.75676132696657, m5_rate=13.616657336513985, m15_rate=14.394546261321954, mean_rate=2.3148080625671974, rate_unit=events/second +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1104, m1_rate=145.560331148258, m5_rate=203.14580675015458, m15_rate=214.7510685472897, mean_rate=34.534186912207815, rate_unit=events/second +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=269.496, max=269.496, mean=269.496, stddev=0.0, p50=269.496, p75=269.496, p95=269.496, p98=269.496, p99=269.496, p999=269.496, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031279512342080346, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=639.33075, max=639.33075, mean=639.33075, stddev=0.0, p50=639.33075, p75=639.33075, p95=639.33075, p98=639.33075, p99=639.33075, p999=639.33075, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127928580120086, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=139.9175, max=139.9175, mean=139.9175, stddev=0.0, p50=139.9175, p75=139.9175, p95=139.9175, p98=139.9175, p99=139.9175, p999=139.9175, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127915502279425, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=223.235042, max=223.235042, mean=223.235042, stddev=0.0, p50=223.235042, p75=223.235042, p95=223.235042, p98=223.235042, p99=223.235042, p999=223.235042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031278966155411835, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:26:20 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:26:20 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:26:20 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:26:20 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:26:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 +18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8604860857340579093 +18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 +18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:23 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:29:23 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 13:29:23 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:29:23 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:29:23 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:29:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:23 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, existing:[] +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 - service started +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8604860857340579093, topicPartition: test-0 +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8604860857340579093. +18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 retrieved 0 file names +18-08-2023 13:29:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093. +18-08-2023 13:29:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 doesn't exist +18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 retrieved 0 file names +18-08-2023 13:29:24 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: cleaner started +18-08-2023 13:29:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8604860857340579093","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390563717} of type:kafka_pipe_start +18-08-2023 13:29:24 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz +18-08-2023 13:29:24 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:29:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093, filePath:TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz +18-08-2023 13:29:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 +18-08-2023 13:29:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093, filePath:TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz +18-08-2023 13:29:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz +18-08-2023 13:29:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz] +18-08-2023 13:29:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz] +18-08-2023 13:29:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0/insertFiles?requestId=1359fcb3-ffcd-4e13-888e-46010696cb74&showSkippedFiles=false +18-08-2023 13:29:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:29:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:29:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:53 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:29:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: cleaner terminated +18-08-2023 13:29:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:29:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:29:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:29:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8604860857340579093","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1362,"average_commit_lag_file_count":1,"start_time":1692390563709,"end_time":1692390597754,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:29:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: service closed +18-08-2023 13:29:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 +18-08-2023 13:29:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 is:0, names:[] +18-08-2023 13:29:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 dropped +18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 dropped +18-08-2023 13:29:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 +18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6937783810298395356 +18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 +18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:58 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:29:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:29:58 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:29:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:29:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, existing:[] +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 - service started +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6937783810298395356, topicPartition: test-0 +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:29:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 +18-08-2023 13:29:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6937783810298395356. +18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 retrieved 0 file names +18-08-2023 13:29:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356. +18-08-2023 13:29:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 doesn't exist +18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 retrieved 0 file names +18-08-2023 13:29:59 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: cleaner started +18-08-2023 13:29:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6937783810298395356","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390598810} of type:kafka_pipe_start +18-08-2023 13:29:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz +18-08-2023 13:29:59 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 13:29:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356, filePath:TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz +18-08-2023 13:29:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 +18-08-2023 13:30:00 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356, filePath:TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz +18-08-2023 13:30:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz +18-08-2023 13:30:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz] +18-08-2023 13:30:00 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz] +18-08-2023 13:30:00 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0/insertFiles?requestId=4f4a34a6-88cf-4d09-8536-db7d4f800c68&showSkippedFiles=false +18-08-2023 13:30:00 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:30:00 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:30:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: cleaner terminated +18-08-2023 13:30:31 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:30:31 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:30:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:30:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6937783810298395356","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":983,"average_commit_lag_file_count":1,"start_time":1692390598807,"end_time":1692390631388,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:30:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: service closed +18-08-2023 13:30:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 +18-08-2023 13:30:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 is:0, names:[] +18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 dropped +18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 dropped +18-08-2023 13:30:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 +18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1197281858695507582 +18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 +18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:32 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:30:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:30:32 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:30:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:30:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, existing:[] +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 - service started +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1197281858695507582, topicPartition: test-0 +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1197281858695507582. +18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 retrieved 0 file names +18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582. +18-08-2023 13:30:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 doesn't exist +18-08-2023 13:30:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:30:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 retrieved 0 file names +18-08-2023 13:30:33 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: cleaner started +18-08-2023 13:30:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1197281858695507582","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390632470} of type:kafka_pipe_start +18-08-2023 13:30:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz +18-08-2023 13:30:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:30:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582, filePath:TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz +18-08-2023 13:30:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 +18-08-2023 13:30:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582, filePath:TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz +18-08-2023 13:30:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz +18-08-2023 13:30:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz] +18-08-2023 13:30:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz] +18-08-2023 13:30:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0/insertFiles?requestId=2ca659be-2980-4719-9763-a479c83040d4&showSkippedFiles=false +18-08-2023 13:30:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:30:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:30:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:30:37 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:42 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:47 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:52 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:30:57 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:02 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: cleaner terminated +18-08-2023 13:31:05 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:31:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:31:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:31:05 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1197281858695507582","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":984,"average_commit_lag_file_count":1,"start_time":1692390632467,"end_time":1692390665074,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:31:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: service closed +18-08-2023 13:31:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 +18-08-2023 13:31:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 is:0, names:[] +18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 dropped +18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 dropped +18-08-2023 13:31:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:31:05 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 +18-08-2023 13:31:05 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3765566255072832355 +18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:31:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 +18-08-2023 13:31:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:31:06 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:31:06 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:31:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:31:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, existing:[] +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 - service started +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3765566255072832355, topicPartition: test-0 +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:31:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:31:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3765566255072832355. +18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 retrieved 0 file names +18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355. +18-08-2023 13:31:06 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 doesn't exist +18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 retrieved 0 file names +18-08-2023 13:31:06 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: cleaner started +18-08-2023 13:31:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3765566255072832355","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390666153} of type:kafka_pipe_start +18-08-2023 13:31:06 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz +18-08-2023 13:31:06 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 13:31:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355, filePath:TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz +18-08-2023 13:31:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 +18-08-2023 13:31:07 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355, filePath:TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz +18-08-2023 13:31:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz +18-08-2023 13:31:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz] +18-08-2023 13:31:07 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz] +18-08-2023 13:31:07 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0/insertFiles?requestId=1e4d8fab-60bc-495a-9788-5cb8b1b71bb3&showSkippedFiles=false +18-08-2023 13:31:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:31:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:31:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:36 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: cleaner terminated +18-08-2023 13:31:40 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:31:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:31:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:31:40 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:40 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3765566255072832355","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2333,"average_commit_lag_file_count":1,"start_time":1692390666140,"end_time":1692390700233,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:31:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: service closed +18-08-2023 13:31:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 +18-08-2023 13:31:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 is:0, names:[] +18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 dropped +18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 dropped +18-08-2023 13:31:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:31:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 +18-08-2023 13:31:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 +18-08-2023 13:31:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:40 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. +18-08-2023 13:31:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:40 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3449896833657992937 +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 +18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 +18-08-2023 13:31:41 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:31:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:31:41 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:31:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:31:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 +18-08-2023 13:31:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0, existing:[] +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 - service started +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3449896833657992937, topicPartition: test-0 +18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 dropped +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 dropped +18-08-2023 13:31:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 +18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 +18-08-2023 13:31:41 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:41 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:41 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:31:41 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_188487355215212289 +18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 +18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 +18-08-2023 13:31:42 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:31:42 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:31:42 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:31:42 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 +18-08-2023 13:31:42 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0, existing:[] +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 - service started +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_188487355215212289, topicPartition: test-0 +18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 dropped +18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 dropped +18-08-2023 13:31:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 +18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:42 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1080103072068495127 +18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:31:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 +18-08-2023 13:31:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:43 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:31:43 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:31:43 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:31:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:43 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, existing:[] +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 - service started +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1080103072068495127, topicPartition: test-0 +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1080103072068495127. +18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 retrieved 0 file names +18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127. +18-08-2023 13:31:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 doesn't exist +18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 retrieved 0 file names +18-08-2023 13:31:43 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: cleaner started +18-08-2023 13:31:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1080103072068495127","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390703132} of type:kafka_pipe_start +18-08-2023 13:31:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz +18-08-2023 13:31:43 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:31:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127, filePath:TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz +18-08-2023 13:31:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 +18-08-2023 13:31:44 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127, filePath:TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz +18-08-2023 13:31:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz +18-08-2023 13:31:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz] +18-08-2023 13:31:44 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz] +18-08-2023 13:31:44 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0/insertFiles?requestId=62226d3c-a83a-4898-80f6-9e8c153b2e77&showSkippedFiles=false +18-08-2023 13:31:44 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:31:44 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:31:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:31:46 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:51 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:31:56 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:01 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:06 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:11 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: cleaner terminated +18-08-2023 13:32:15 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:32:15 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:32:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:32:15 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1080103072068495127","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":937,"average_commit_lag_file_count":1,"start_time":1692390703131,"end_time":1692390735535,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:32:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: service closed +18-08-2023 13:32:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 +18-08-2023 13:32:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 is:0, names:[] +18-08-2023 13:32:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 dropped +18-08-2023 13:32:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 dropped +18-08-2023 13:32:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 +18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:16 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8516722832153005043 +18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 +18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:16 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:32:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:32:16 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:32:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:32:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, existing:[] +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 - service started +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8516722832153005043, topicPartition: test-0 +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:32:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:32:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8516722832153005043. +18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 retrieved 0 file names +18-08-2023 13:32:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043. +18-08-2023 13:32:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 doesn't exist +18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 retrieved 0 file names +18-08-2023 13:32:17 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: cleaner started +18-08-2023 13:32:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8516722832153005043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390736805} of type:kafka_pipe_start +18-08-2023 13:32:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz +18-08-2023 13:32:17 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 +18-08-2023 13:32:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043, filePath:TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz +18-08-2023 13:32:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 +18-08-2023 13:32:18 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043, filePath:TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz +18-08-2023 13:32:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz +18-08-2023 13:32:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz] +18-08-2023 13:32:18 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz] +18-08-2023 13:32:18 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0/insertFiles?requestId=5bcf3187-a900-43bb-adc0-c396d49c2b9e&showSkippedFiles=false +18-08-2023 13:32:19 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:32:19 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:32:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:21 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:26 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:31 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:36 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:41 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:46 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: cleaner terminated +18-08-2023 13:32:50 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:32:50 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:32:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:32:50 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:32:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8516722832153005043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1196,"average_commit_lag_file_count":1,"start_time":1692390736804,"end_time":1692390770337,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:32:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: service closed +18-08-2023 13:32:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 +18-08-2023 13:32:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 is:0, names:[] +18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 dropped +18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 dropped +18-08-2023 13:32:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 +18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2406829002217351794 +18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 +18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:51 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:32:51 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:32:51 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:32:51 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:32:51 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:51 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, existing:[] +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 - service started +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2406829002217351794, topicPartition: test-0 +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2406829002217351794. +18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 retrieved 0 file names +18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794. +18-08-2023 13:32:51 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 doesn't exist +18-08-2023 13:32:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:32:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 retrieved 0 file names +18-08-2023 13:32:52 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2406829002217351794","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390771389} of type:kafka_pipe_start +18-08-2023 13:32:52 pool-28-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: cleaner started +18-08-2023 13:32:52 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz +18-08-2023 13:32:52 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 13:32:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794, filePath:TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz +18-08-2023 13:32:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 +18-08-2023 13:32:52 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794, filePath:TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz +18-08-2023 13:32:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz +18-08-2023 13:32:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz] +18-08-2023 13:32:52 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz] +18-08-2023 13:32:52 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0/insertFiles?requestId=1f106c8b-e19e-4dc9-b159-315d34768c1d&showSkippedFiles=false +18-08-2023 13:32:53 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:32:53 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:32:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:32:56 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:01 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:06 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:11 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:16 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:21 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: cleaner terminated +18-08-2023 13:33:23 pool-28-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:33:23 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:33:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:33:23 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:23 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2406829002217351794","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":920,"average_commit_lag_file_count":1,"start_time":1692390771388,"end_time":1692390803907,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:33:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: service closed +18-08-2023 13:33:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 +18-08-2023 13:33:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 is:0, names:[] +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 dropped +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 dropped +18-08-2023 13:33:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 +18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7584709008059395876 +18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 +18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:24 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:33:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:33:24 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:33:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 13:33:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, existing:[] +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 - service started +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7584709008059395876, topicPartition: test-0 +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:33:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:33:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7584709008059395876. +18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 retrieved 0 file names +18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876. +18-08-2023 13:33:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 doesn't exist +18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 retrieved 0 file names +18-08-2023 13:33:25 pool-31-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: cleaner started +18-08-2023 13:33:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7584709008059395876","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390804775} of type:kafka_pipe_start +18-08-2023 13:33:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz +18-08-2023 13:33:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 +18-08-2023 13:33:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876, filePath:TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz +18-08-2023 13:33:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 +18-08-2023 13:33:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876, filePath:TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz +18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz +18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz] +18-08-2023 13:33:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz] +18-08-2023 13:33:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0/insertFiles?requestId=b773c5a1-ba13-4936-96f5-77f33e3aeac4&showSkippedFiles=false +18-08-2023 13:33:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:33:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 13:33:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:29 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:34 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:39 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:49 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:54 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: cleaner terminated +18-08-2023 13:33:57 pool-31-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 13:33:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:33:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 13:33:57 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:33:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7584709008059395876","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1000,"average_commit_lag_file_count":1,"start_time":1692390804774,"end_time":1692390837339,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 13:33:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: service closed +18-08-2023 13:33:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 +18-08-2023 13:33:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 is:0, names:[] +18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 dropped +18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 dropped +18-08-2023 13:33:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:57 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:33:58 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 13:33:58 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:33:58 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:33:58 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:58 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:33:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:33:58 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:33:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:33:58 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:33:58 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:33:58 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:33:58 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1063988221374408804 doesn't exist +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1063988221374408804. +18-08-2023 13:33:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1063988221374408804 +18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1063988221374408804 +18-08-2023 13:33:58 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1063988221374408804, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_0","table":"kafka_connector_test_table_1063988221374408804"} +18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1063988221374408804, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804 +18-08-2023 13:33:58 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset:null +18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1063988221374408804, topicPartition: test-0 +18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 13:33:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:33:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1050 +18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 +18-08-2023 13:33:59 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1f098399[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec] +18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, idx=0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. +18-08-2023 13:33:59 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1105: footer length = 755 +18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 755 => 243 2 0 0 +18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1113, encryptedCompressedSize=1120, bdecVersion=THREE +18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, size=1120 +18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, size=1120, timeInMillis=204 +18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, idx=0 +18-08-2023 13:33:59 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 13:33:59 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 13:33:59 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec","md5":"4e1696ba5678fcfc54c3c6cfb2e17112","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","chunk_start_offset":0,"chunk_length":1113,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"f52d2bc6308888e944c18a30ac6ce963","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370837370,"first_insert_time_in_ms":1692390838783,"last_insert_time_in_ms":1692390838783,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390839246,"build_duration_ms":260,"upload_duration_ms":202},"bdec_version":3}],"role":"testrole_kafka","request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_2"} +18-08-2023 13:33:59 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 13:34:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 13:34:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:34:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:34:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:03 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:34:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:34:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 13:34:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 13:34:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 +18-08-2023 13:34:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:08 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:34:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:34:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:34:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 13:34:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:34:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:13 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:34:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1027 +18-08-2023 13:34:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 13:34:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 13:34:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:34:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:18 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 13:34:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 13:34:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 13:34:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 13:34:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:23 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:34:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:34:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 13:34:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 13:34:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:34:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:28 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:34:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1055, currentDiffMillis=1055 +18-08-2023 13:34:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 +18-08-2023 13:34:29 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:34:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset:0 +18-08-2023 13:34:29 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 13:34:29 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:34:29 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +18-08-2023 13:34:29 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 13:34:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=407, currentDiffMillis=407 +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=23, mean=0.3032728552534317, stddev=2.064925961469176, p50=0.0, p75=0.0, p95=0.0, p98=3.0, p99=13.0, p999=23.0 +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.454872103818902, rate_unit=events/second +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.24924890737557, rate_unit=events/second +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=260.722042, max=260.722042, mean=260.722042, stddev=0.0, p50=260.722042, p75=260.722042, p95=260.722042, p98=260.722042, p99=260.722042, p999=260.722042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147083486471098, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=648.610417, max=648.610417, mean=648.610417, stddev=0.0, p50=648.610417, p75=648.610417, p95=648.610417, p98=648.610417, p99=648.610417, p999=648.610417, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147054438606101, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=181.051959, max=181.051959, mean=181.051959, stddev=0.0, p50=181.051959, p75=181.051959, p95=181.051959, p98=181.051959, p99=181.051959, p999=181.051959, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147037746475505, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=202.078459, max=202.078459, mean=202.078459, stddev=0.0, p50=202.078459, p75=202.078459, p95=202.078459, p98=202.078459, p99=202.078459, p999=202.078459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031470203817609614, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:34:29 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:34:29 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:34:29 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 13:34:29 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:30 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:34:30 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 13:34:30 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:34:30 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:30 Thread-18 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:34:30 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:34:30 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:34:30 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:34:30 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:34:30 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:34:30 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:34:30 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3694719056936488009 doesn't exist +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3694719056936488009. +18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3694719056936488009 +18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_3694719056936488009 +18-08-2023 13:34:30 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3694719056936488009, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_0","table":"kafka_connector_test_table_3694719056936488009"} +18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3694719056936488009, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009 +18-08-2023 13:34:30 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset:null +18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3694719056936488009, topicPartition: test-0 +18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:34:30 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:34:30 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} +18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 +18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:34:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1047 +18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 +18-08-2023 13:34:31 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@71d66860[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, idx=0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1154: footer length = 774 +18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 774 => 6 3 0 0 +18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1162, encryptedCompressedSize=1168, bdecVersion=THREE +18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, size=1168 +18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, size=1168, timeInMillis=286 +18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, idx=0 +18-08-2023 13:34:31 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 13:34:31 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 13:34:31 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec","md5":"2b688fb4ea514dd03c46115d1cbe3c77","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","chunk_start_offset":0,"chunk_length":1162,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"bd1e6dd7bd9625d6d138caa1d75c0c74","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370836294,"first_insert_time_in_ms":1692390870947,"last_insert_time_in_ms":1692390870947,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390871367,"build_duration_ms":7,"upload_duration_ms":285},"bdec_version":3}],"role":"testrole_kafka","request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_2"} +18-08-2023 13:34:31 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 13:34:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 13:34:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 +18-08-2023 13:34:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1051 +18-08-2023 13:34:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:35 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:34:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 13:34:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:34:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 13:34:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1040 +18-08-2023 13:34:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:40 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 +18-08-2023 13:34:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 13:34:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 13:34:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +18-08-2023 13:34:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 +18-08-2023 13:34:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:45 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1018, currentDiffMillis=1019 +18-08-2023 13:34:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 +18-08-2023 13:34:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 13:34:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 13:34:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 +18-08-2023 13:34:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:50 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 13:34:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 13:34:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 +18-08-2023 13:34:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:34:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 +18-08-2023 13:34:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:55 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:34:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +18-08-2023 13:34:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 13:34:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 13:34:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:34:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +18-08-2023 13:34:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:35:00 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 +18-08-2023 13:35:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:35:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 13:35:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:35:01 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 +18-08-2023 13:35:01 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:01 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset:1 +18-08-2023 13:35:01 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 13:35:01 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 13:35:01 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:02 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 13:35:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=598, currentDiffMillis=598 +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=10, mean=0.12375958740937146, stddev=0.9474530713527652, p50=0.0, p75=0.0, p95=0.0, p98=1.0, p99=7.0, p999=10.0 +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.611643738283248, rate_unit=events/second +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.75162256801035, rate_unit=events/second +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.548916, max=7.548916, mean=7.548916, stddev=0.0, p50=7.548916, p75=7.548916, p95=7.548916, p98=7.548916, p99=7.548916, p999=7.548916, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146518957110048, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=449.222917, max=449.222917, mean=449.222917, stddev=0.0, p50=449.222917, p75=449.222917, p95=449.222917, p98=449.222917, p99=449.222917, p999=449.222917, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146496874809927, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=154.766375, max=154.766375, mean=154.766375, stddev=0.0, p50=154.766375, p75=154.766375, p95=154.766375, p98=154.766375, p99=154.766375, p999=154.766375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146477119491828, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=285.631042, max=285.631042, mean=285.631042, stddev=0.0, p50=285.631042, p75=285.631042, p95=285.631042, p98=285.631042, p99=285.631042, p999=285.631042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146453490876679, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:35:02 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 13:35:02 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:02 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:35:02 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:02 Thread-20 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:35:02 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:35:02 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:35:02 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:35:02 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:35:02 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 +18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:35:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:35:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:35:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_303740236573408996 doesn't exist +18-08-2023 13:35:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_303740236573408996. +18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_303740236573408996 +18-08-2023 13:35:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_303740236573408996 +18-08-2023 13:35:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_303740236573408996, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_0","table":"kafka_connector_test_table_303740236573408996"} +18-08-2023 13:35:03 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_303740236573408996, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:03 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_303740236573408996 +18-08-2023 13:35:03 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:03 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset:null +18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:35:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_303740236573408996, topicPartition: test-0 +18-08-2023 13:35:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 13:35:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 +18-08-2023 13:35:03 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@14324c4[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, idx=0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1103: footer length = 753 +18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 753 => 241 2 0 0 +18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1111, encryptedCompressedSize=1120, bdecVersion=THREE +18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, size=1120 +18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, size=1120, timeInMillis=206 +18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, idx=0 +18-08-2023 13:35:03 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 13:35:03 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +18-08-2023 13:35:03 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec","md5":"80778765d8082a2441ab0b595be2db4a","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","chunk_start_offset":0,"chunk_length":1111,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"066a278cb6ecd399b6a10a550f4ef02e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370838602,"first_insert_time_in_ms":1692390903266,"last_insert_time_in_ms":1692390903266,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390903690,"build_duration_ms":7,"upload_duration_ms":206},"bdec_version":3}],"role":"testrole_kafka","request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_2"} +18-08-2023 13:35:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 +18-08-2023 13:35:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:35:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +18-08-2023 13:35:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:35:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:07 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 13:35:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:35:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1026 +18-08-2023 13:35:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1049 +18-08-2023 13:35:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 13:35:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:12 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:35:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1050 +18-08-2023 13:35:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 13:35:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 +18-08-2023 13:35:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 13:35:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:17 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 13:35:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 +18-08-2023 13:35:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 13:35:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:35:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 13:35:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:22 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 13:35:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:35:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 13:35:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +18-08-2023 13:35:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 13:35:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:27 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 +18-08-2023 13:35:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 13:35:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:35:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:35:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:32 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 13:35:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1027 +18-08-2023 13:35:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:33 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 +18-08-2023 13:35:33 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset:0 +18-08-2023 13:35:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 13:35:34 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:34 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 13:35:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=380, currentDiffMillis=380 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=305, min=0, max=5, mean=0.01961205603582848, stddev=0.2667514544940201, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=0.0, p999=5.0 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4684269534666727, rate_unit=events/second +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.44391070583461, rate_unit=events/second +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.464542, max=7.464542, mean=7.464542, stddev=0.0, p50=7.464542, p75=7.464542, p95=7.464542, p98=7.464542, p99=7.464542, p999=7.464542, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164611908801335, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=369.784375, max=369.784375, mean=369.784375, stddev=0.0, p50=369.784375, p75=369.784375, p95=369.784375, p98=369.784375, p99=369.784375, p999=369.784375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164592133863106, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=154.738667, max=154.738667, mean=154.738667, stddev=0.0, p50=154.738667, p75=154.738667, p95=154.738667, p98=154.738667, p99=154.738667, p999=154.738667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164574358071113, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=206.072875, max=206.072875, mean=206.072875, stddev=0.0, p50=206.072875, p75=206.072875, p95=206.072875, p98=206.072875, p99=206.072875, p999=206.072875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031645555266590394, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:35:34 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:34 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 +18-08-2023 13:35:34 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:35:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:35:34 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:34 Thread-22 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:35:34 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:35:34 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:35:34 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:35:34 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:35:34 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:35:34 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 +18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:35:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:35:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:35:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:35:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1456398297857472324 doesn't exist +18-08-2023 13:35:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1456398297857472324. +18-08-2023 13:35:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1456398297857472324 +18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1456398297857472324 +18-08-2023 13:35:35 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1456398297857472324, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:35:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_0","table":"kafka_connector_test_table_1456398297857472324"} +18-08-2023 13:35:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1456398297857472324, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:35:35 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324 +18-08-2023 13:35:35 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset:null +18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:35:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1456398297857472324, topicPartition: test-0 +18-08-2023 13:35:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:35:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 13:35:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} +18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 +18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 13:35:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:35:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 +18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 +18-08-2023 13:35:35 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1aecae47[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec] +18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, idx=0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1112: footer length = 750 +18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 750 => 238 2 0 0 +18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1120, encryptedCompressedSize=1136, bdecVersion=THREE +18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, size=1136 +18-08-2023 13:35:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, size=1136, timeInMillis=212 +18-08-2023 13:35:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, idx=0 +18-08-2023 13:35:36 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 13:35:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +18-08-2023 13:35:36 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec","md5":"b2c1fc2deecfd9f1067a1271ff16acce","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","chunk_start_offset":0,"chunk_length":1120,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"6e99914afdcd9f57663286e65d3a4347","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370841702,"first_insert_time_in_ms":1692390935590,"last_insert_time_in_ms":1692390935590,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390935887,"build_duration_ms":7,"upload_duration_ms":211},"bdec_version":3}],"role":"testrole_kafka","request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_2"} +18-08-2023 13:35:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 +18-08-2023 13:35:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 +18-08-2023 13:35:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 13:35:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 +18-08-2023 13:35:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:39 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 13:35:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 13:35:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 13:35:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:35:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1023, currentDiffMillis=1023 +18-08-2023 13:35:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:44 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 13:35:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 13:35:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 +18-08-2023 13:35:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 13:35:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:35:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:49 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 13:35:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 13:35:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 +18-08-2023 13:35:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 13:35:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 13:35:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:54 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:35:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 13:35:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 13:35:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 13:35:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 13:35:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:35:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:35:59 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:36:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 13:36:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 +18-08-2023 13:36:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 +18-08-2023 13:36:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 +18-08-2023 13:36:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:04 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:36:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1054, currentDiffMillis=1055 +18-08-2023 13:36:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 13:36:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:06 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 +18-08-2023 13:36:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:36:06 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset:1 +18-08-2023 13:36:06 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 13:36:06 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:36:06 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 13:36:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 13:36:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=616, currentDiffMillis=616 +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=4, mean=0.038359723140007185, stddev=0.3263378480837562, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=1.0, p999=4.0 +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.4180631262543066, rate_unit=events/second +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.67416513032052, rate_unit=events/second +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.022, max=7.022, mean=7.022, stddev=0.0, p50=7.022, p75=7.022, p95=7.022, p98=7.022, p99=7.022, p999=7.022, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0314031454534754, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=377.704209, max=377.704209, mean=377.704209, stddev=0.0, p50=377.704209, p75=377.704209, p95=377.704209, p98=377.704209, p99=377.704209, p999=377.704209, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03140296950719061, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=157.849375, max=157.849375, mean=157.849375, stddev=0.0, p50=157.849375, p75=157.849375, p95=157.849375, p98=157.849375, p99=157.849375, p999=157.849375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03140286830423283, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=211.400459, max=211.400459, mean=211.400459, stddev=0.0, p50=211.400459, p75=211.400459, p95=211.400459, p98=211.400459, p99=211.400459, p999=211.400459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031402734846510606, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 13:36:06 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:36:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 13:36:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 +18-08-2023 13:36:06 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 13:36:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:36:07 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:07 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:07 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:07 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:36:07 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:36:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:36:07 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:07 Thread-24 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 13:36:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:36:07 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:36:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:36:07 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_4 +18-08-2023 13:36:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:36:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_4, total available processors=10 +18-08-2023 13:36:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_4, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:36:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_4 +18-08-2023 13:36:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:36:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:36:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:36:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1976254417471840300 doesn't exist +18-08-2023 13:36:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1976254417471840300. +18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1976254417471840300 +18-08-2023 13:36:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1976254417471840300 +18-08-2023 13:36:07 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1976254417471840300, client=KC_CLIENT_TEST_CONNECTOR_4 +18-08-2023 13:36:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"J70P5icEwn0OWxlmpzOmSOVQSAA7WNiBmGZjnMm6gINkCC_1003_0","table":"kafka_connector_test_table_1976254417471840300"} +18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1976254417471840300, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_4 +18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300 +18-08-2023 13:36:08 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:36:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 +18-08-2023 13:36:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"J70P5icEwn0OWxlmpzOmSOVQSAA7WNiBmGZjnMm6gINkCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:36:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, offset:null +18-08-2023 13:36:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1976254417471840300, topicPartition: test-0 +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_4, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 13:36:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 +18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 13:36:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 13:36:08 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:08 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:08 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 13:36:08 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 13:36:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 13:36:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 13:36:08 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 13:36:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 13:36:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_5 +18-08-2023 13:36:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 13:36:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_5, total available processors=10 +18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_5, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 13:36:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_5 +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 13:36:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4342516535815138887 doesn't exist +18-08-2023 13:36:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4342516535815138887. +18-08-2023 13:36:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4342516535815138887 +18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4342516535815138887 +18-08-2023 13:36:09 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4342516535815138887, client=KC_CLIENT_TEST_CONNECTOR_5 +18-08-2023 13:36:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"d4D0b97qlayku4mqQP6JtTo3IQe586Y3NeSf5RJ1QBBUCC_1003_0","table":"kafka_connector_test_table_4342516535815138887"} +18-08-2023 13:36:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4342516535815138887, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_5 +18-08-2023 13:36:09 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887 +18-08-2023 13:36:09 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 13:36:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0 +18-08-2023 13:36:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"d4D0b97qlayku4mqQP6JtTo3IQe586Y3NeSf5RJ1QBBUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, offset:null +18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 13:36:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4342516535815138887, topicPartition: test-0 +18-08-2023 13:36:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 13:36:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 13:36:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_4, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 13:36:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 +18-08-2023 14:38:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:38:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 +18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 +18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5224312851784726463 +18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 +18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 +18-08-2023 14:38:15 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:38:15 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:38:15 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:38:15 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:38:15 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:38:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 +18-08-2023 14:38:15 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0, existing:[] +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 - service started +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5224312851784726463, topicPartition: test-0 +18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 dropped +18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 dropped +18-08-2023 14:38:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:38:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 +18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7794421424625032973 +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 +18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 +18-08-2023 14:38:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:38:16 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:38:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:38:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 +18-08-2023 14:38:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0, existing:[] +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 - service started +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7794421424625032973, topicPartition: test-0 +18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 dropped +18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 dropped +18-08-2023 14:38:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:39:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 +18-08-2023 14:39:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 +18-08-2023 14:39:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2657774367744592480 +18-08-2023 14:39:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:39:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 +18-08-2023 14:39:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 +18-08-2023 14:39:31 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:39:31 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:39:31 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:39:31 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:39:31 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:39:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 +18-08-2023 14:39:31 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0, existing:[] +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 - service started +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2657774367744592480, topicPartition: test-0 +18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:39:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 14:39:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 dropped +18-08-2023 14:39:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 dropped +18-08-2023 14:39:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 14:39:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 +18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 +18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3366560785811907597 +18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 +18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 +18-08-2023 14:39:55 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:39:55 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:39:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:39:55 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:39:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:39:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 +18-08-2023 14:39:55 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0, existing:[] +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 - service started +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3366560785811907597, topicPartition: test-0 +18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:40:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 14:40:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:40:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 +18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 +18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6286529608848783160 +18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 +18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 +18-08-2023 14:40:10 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:40:10 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:40:10 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:40:10 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:40:10 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:40:10 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 +18-08-2023 14:40:10 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0, existing:[] +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 - service started +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6286529608848783160, topicPartition: test-0 +18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1644140466561349223 +18-08-2023 14:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1644140466561349223_0 +18-08-2023 14:42:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:42:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:42:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:42:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2821288052890949006 +18-08-2023 14:42:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2821288052890949006_0 +18-08-2023 14:42:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:42:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:42:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 +18-08-2023 14:42:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:42 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:42:42 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:42:42 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:42:42 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3121063502165281348 +18-08-2023 14:42:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:42:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 +18-08-2023 14:42:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:43 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:42:43 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:42:43 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:42:43 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:42:43 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:42:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:43 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, existing:[] +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 - service started +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3121063502165281348, topicPartition: test-0 +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:42:43 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 14:42:43 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3121063502165281348. +18-08-2023 14:42:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 retrieved 0 file names +18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348. +18-08-2023 14:42:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 doesn't exist +18-08-2023 14:42:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:42:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 retrieved 0 file names +18-08-2023 14:42:44 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: cleaner started +18-08-2023 14:42:44 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3121063502165281348","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692394963406} of type:kafka_pipe_start +18-08-2023 14:42:44 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz +18-08-2023 14:42:44 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 +18-08-2023 14:42:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348, filePath:TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz +18-08-2023 14:42:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 +18-08-2023 14:42:44 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348, filePath:TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz +18-08-2023 14:42:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz +18-08-2023 14:42:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz] +18-08-2023 14:42:44 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz] +18-08-2023 14:42:44 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0/insertFiles?requestId=d5b59727-3ec7-4059-a245-b1b21238cc8c&showSkippedFiles=false +18-08-2023 14:42:45 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:42:45 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:42:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:42:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:42:53 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:42:58 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:13 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: cleaner terminated +18-08-2023 14:43:16 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:43:16 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:43:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:43:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:16 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3121063502165281348","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1329,"average_commit_lag_file_count":1,"start_time":1692394963367,"end_time":1692394996326,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:43:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: service closed +18-08-2023 14:43:16 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 +18-08-2023 14:43:16 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 is:0, names:[] +18-08-2023 14:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 dropped +18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 dropped +18-08-2023 14:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 +18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:17 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:17 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:17 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:17 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6727268077568408945 +18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 +18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:17 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:43:17 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 14:43:17 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:43:17 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:43:17 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:43:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:17 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, existing:[] +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 - service started +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6727268077568408945, topicPartition: test-0 +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6727268077568408945. +18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 retrieved 0 file names +18-08-2023 14:43:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945. +18-08-2023 14:43:18 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 doesn't exist +18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 retrieved 0 file names +18-08-2023 14:43:18 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: cleaner started +18-08-2023 14:43:18 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6727268077568408945","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692394997676} of type:kafka_pipe_start +18-08-2023 14:43:18 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz +18-08-2023 14:43:18 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 14:43:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945, filePath:TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz +18-08-2023 14:43:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 +18-08-2023 14:43:19 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945, filePath:TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz +18-08-2023 14:43:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz +18-08-2023 14:43:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz] +18-08-2023 14:43:19 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz] +18-08-2023 14:43:19 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0/insertFiles?requestId=72341438-dc1f-4590-8d68-6493546bd830&showSkippedFiles=false +18-08-2023 14:43:19 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:43:19 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:43:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:50 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:43:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: cleaner terminated +18-08-2023 14:43:50 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:43:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:43:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:43:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6727268077568408945","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1308,"average_commit_lag_file_count":1,"start_time":1692394997668,"end_time":1692395030429,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:43:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: service closed +18-08-2023 14:43:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 +18-08-2023 14:43:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 is:0, names:[] +18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 dropped +18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 dropped +18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 +18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:51 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:51 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:51 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:43:51 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_152605127061635629 +18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 +18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:51 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:43:51 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:43:51 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:43:51 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:43:51 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:51 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, existing:[] +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 - service started +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_152605127061635629, topicPartition: test-0 +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:43:51 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 14:43:51 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_152605127061635629. +18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 retrieved 0 file names +18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629. +18-08-2023 14:43:51 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 doesn't exist +18-08-2023 14:43:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:43:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 retrieved 0 file names +18-08-2023 14:43:52 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: cleaner started +18-08-2023 14:43:52 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_152605127061635629","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395031532} of type:kafka_pipe_start +18-08-2023 14:43:52 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz +18-08-2023 14:43:52 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 +18-08-2023 14:43:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629, filePath:TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz +18-08-2023 14:43:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 +18-08-2023 14:43:52 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629, filePath:TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz +18-08-2023 14:43:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz +18-08-2023 14:43:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz] +18-08-2023 14:43:52 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz] +18-08-2023 14:43:52 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0/insertFiles?requestId=f23cec22-e7d4-436c-8074-976667a7d8ed&showSkippedFiles=false +18-08-2023 14:43:53 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:43:53 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:43:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:43:56 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:01 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:06 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: cleaner terminated +18-08-2023 14:44:24 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:44:24 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:44:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:44:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_152605127061635629","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1131,"average_commit_lag_file_count":1,"start_time":1692395031480,"end_time":1692395064218,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:44:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: service closed +18-08-2023 14:44:24 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 +18-08-2023 14:44:24 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 is:0, names:[] +18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 dropped +18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 dropped +18-08-2023 14:44:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:44:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 +18-08-2023 14:44:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:24 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:24 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:24 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:24 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_445993476782435834 +18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:44:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 +18-08-2023 14:44:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:25 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:44:25 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:44:25 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:44:25 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:44:25 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:25 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, existing:[] +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 - service started +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_445993476782435834, topicPartition: test-0 +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_445993476782435834. +18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 retrieved 0 file names +18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834. +18-08-2023 14:44:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 doesn't exist +18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 retrieved 0 file names +18-08-2023 14:44:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_445993476782435834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395065101} of type:kafka_pipe_start +18-08-2023 14:44:25 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: cleaner started +18-08-2023 14:44:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz +18-08-2023 14:44:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 14:44:26 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834, filePath:TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz +18-08-2023 14:44:26 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 +18-08-2023 14:44:26 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834, filePath:TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz +18-08-2023 14:44:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz +18-08-2023 14:44:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz] +18-08-2023 14:44:26 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz] +18-08-2023 14:44:26 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0/insertFiles?requestId=1b6fd0b8-81a9-4bd8-81b5-96c425eeed88&showSkippedFiles=false +18-08-2023 14:44:27 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:44:27 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:44:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:30 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:35 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:40 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:45 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:50 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:55 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: cleaner terminated +18-08-2023 14:44:58 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:44:58 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:44:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:44:58 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:44:58 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_445993476782435834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1672,"average_commit_lag_file_count":1,"start_time":1692395065100,"end_time":1692395098295,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:44:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: service closed +18-08-2023 14:44:58 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 +18-08-2023 14:44:58 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 is:0, names:[] +18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 dropped +18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 dropped +18-08-2023 14:44:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:44:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 +18-08-2023 14:44:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:44:58 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:58 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:58 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:44:58 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1398553495658597523 +18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:44:59 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 +18-08-2023 14:44:59 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:44:59 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:44:59 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:44:59 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:44:59 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:44:59 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:44:59 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, existing:[] +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 - service started +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1398553495658597523, topicPartition: test-0 +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:44:59 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 14:44:59 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1398553495658597523. +18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 retrieved 0 file names +18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523. +18-08-2023 14:44:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 doesn't exist +18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:45:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 retrieved 0 file names +18-08-2023 14:45:00 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: cleaner started +18-08-2023 14:45:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1398553495658597523","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395099302} of type:kafka_pipe_start +18-08-2023 14:45:00 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz +18-08-2023 14:45:00 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 +18-08-2023 14:45:00 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523, filePath:TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz +18-08-2023 14:45:00 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 +18-08-2023 14:45:00 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523, filePath:TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz +18-08-2023 14:45:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz +18-08-2023 14:45:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz] +18-08-2023 14:45:00 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz] +18-08-2023 14:45:00 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0/insertFiles?requestId=cdef9147-086d-4024-b7bc-150893b30e49&showSkippedFiles=false +18-08-2023 14:45:01 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:45:01 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:45:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: cleaner terminated +18-08-2023 14:45:32 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:45:32 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:45:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:45:32 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1398553495658597523","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1292,"average_commit_lag_file_count":1,"start_time":1692395099300,"end_time":1692395132217,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:45:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: service closed +18-08-2023 14:45:32 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 +18-08-2023 14:45:32 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 is:0, names:[] +18-08-2023 14:45:32 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 dropped +18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 dropped +18-08-2023 14:45:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:45:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 +18-08-2023 14:45:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:32 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:45:32 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:45:32 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:45:32 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8503004783070881087 +18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:45:33 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 +18-08-2023 14:45:33 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:33 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:45:33 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:45:33 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:45:33 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:45:33 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:33 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, existing:[] +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 - service started +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8503004783070881087, topicPartition: test-0 +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8503004783070881087. +18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 retrieved 0 file names +18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087. +18-08-2023 14:45:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 doesn't exist +18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 retrieved 0 file names +18-08-2023 14:45:33 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: cleaner started +18-08-2023 14:45:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8503004783070881087","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395133115} of type:kafka_pipe_start +18-08-2023 14:45:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz +18-08-2023 14:45:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 +18-08-2023 14:45:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087, filePath:TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz +18-08-2023 14:45:34 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 +18-08-2023 14:45:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087, filePath:TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz +18-08-2023 14:45:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz +18-08-2023 14:45:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz] +18-08-2023 14:45:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz] +18-08-2023 14:45:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0/insertFiles?requestId=c6f95f5c-fcb7-4337-8eba-756935129d41&showSkippedFiles=false +18-08-2023 14:45:35 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:45:35 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:45:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:45:38 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:43 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:48 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:53 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:45:58 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:03 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: cleaner terminated +18-08-2023 14:46:05 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:46:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:46:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:46:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8503004783070881087","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1422,"average_commit_lag_file_count":1,"start_time":1692395133115,"end_time":1692395166003,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:46:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: service closed +18-08-2023 14:46:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 +18-08-2023 14:46:06 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 is:0, names:[] +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 dropped +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 dropped +18-08-2023 14:46:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 +18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:06 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:06 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:06 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:06 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1712531299485417574 +18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 +18-08-2023 14:46:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created +18-08-2023 14:46:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 +18-08-2023 14:46:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 +18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:46:06 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:46:07 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:46:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN +18-08-2023 14:46:07 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:07 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, existing:[] +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 - service started +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1712531299485417574, topicPartition: test-0 +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:46:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 14:46:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1712531299485417574. +18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 retrieved 0 file names +18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574. +18-08-2023 14:46:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 doesn't exist +18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 retrieved 0 file names +18-08-2023 14:46:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1712531299485417574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395167010} of type:kafka_pipe_start +18-08-2023 14:46:07 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: cleaner started +18-08-2023 14:46:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz +18-08-2023 14:46:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 +18-08-2023 14:46:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574, filePath:TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz +18-08-2023 14:46:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 +18-08-2023 14:46:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574, filePath:TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz +18-08-2023 14:46:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz +18-08-2023 14:46:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz] +18-08-2023 14:46:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz] +18-08-2023 14:46:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0/insertFiles?requestId=5ac8ab08-9436-4d32-b745-3d4e89be2d31&showSkippedFiles=false +18-08-2023 14:46:08 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:46:08 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null +18-08-2023 14:46:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:12 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:17 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:22 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:27 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:32 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:37 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: cleaner terminated +18-08-2023 14:46:39 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: +sleep interrupted +18-08-2023 14:46:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:46:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed +18-08-2023 14:46:39 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1712531299485417574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1037,"average_commit_lag_file_count":1,"start_time":1692395167009,"end_time":1692395199721,"is_pipe_closing":true} of type:kafka_pipe_usage +18-08-2023 14:46:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: service closed +18-08-2023 14:46:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 +18-08-2023 14:46:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 is:0, names:[] +18-08-2023 14:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 dropped +18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 dropped +18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:40 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:46:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:46:40 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:46:40 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 14:46:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:40 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:46:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:46:40 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:46:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 14:46:40 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:46:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 14:46:40 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 +18-08-2023 14:46:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 14:46:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:46:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 14:46:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 14:46:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:46:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3234142018577459629 doesn't exist +18-08-2023 14:46:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3234142018577459629. +18-08-2023 14:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3234142018577459629 +18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_3234142018577459629 +18-08-2023 14:46:41 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3234142018577459629, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:46:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_0","table":"kafka_connector_test_table_3234142018577459629"} +18-08-2023 14:46:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3234142018577459629, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:46:41 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629 +18-08-2023 14:46:41 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset:null +18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 14:46:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3234142018577459629, topicPartition: test-0 +18-08-2023 14:46:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} +18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 +18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 14:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:46:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 +18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 +18-08-2023 14:46:41 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@71d00f62[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec] +18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, idx=0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. +18-08-2023 14:46:41 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +18-08-2023 14:46:41 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters +18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end +18-08-2023 14:46:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1102: footer length = 752 +18-08-2023 14:46:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 752 => 240 2 0 0 +18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1110, encryptedCompressedSize=1120, bdecVersion=THREE +18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, size=1120 +18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, size=1120, timeInMillis=227 +18-08-2023 14:46:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, idx=0 +18-08-2023 14:46:42 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 14:46:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 14:46:42 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec","md5":"2a4347b46d9af90d096205cb144abfbf","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","chunk_start_offset":0,"chunk_length":1110,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"763fcfa1d59c43f1250c1b36ae390cc6","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370973898,"first_insert_time_in_ms":1692395201726,"last_insert_time_in_ms":1692395201726,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692395201762,"build_duration_ms":360,"upload_duration_ms":217},"bdec_version":3}],"role":"testrole_kafka","request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_2"} +18-08-2023 14:46:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 +18-08-2023 14:46:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 14:46:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 +18-08-2023 14:46:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 14:46:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:45 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1042 +18-08-2023 14:46:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 +18-08-2023 14:46:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 +18-08-2023 14:46:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 +18-08-2023 14:46:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 14:46:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:50 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 14:46:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 14:46:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 +18-08-2023 14:46:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 +18-08-2023 14:46:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 +18-08-2023 14:46:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:55 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:46:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 14:46:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 14:46:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 14:46:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:46:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 14:46:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 14:47:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:00 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 +18-08-2023 14:47:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 14:47:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 +18-08-2023 14:47:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 14:47:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:05 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 +18-08-2023 14:47:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 14:47:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 14:47:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 14:47:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 14:47:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:10 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 +18-08-2023 14:47:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +18-08-2023 14:47:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:12 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 +18-08-2023 14:47:12 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:47:12 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset:0 +18-08-2023 14:47:12 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 14:47:12 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:47:12 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 +18-08-2023 14:47:12 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 14:47:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=805, currentDiffMillis=805 +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=18, mean=0.23795522761607765, stddev=1.7584974094692836, p50=0.0, p75=0.0, p95=0.0, p98=3.0, p99=16.0, p999=18.0 +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.422997937709851, rate_unit=events/second +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=34.79121362272457, rate_unit=events/second +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=360.579375, max=360.579375, mean=360.579375, stddev=0.0, p50=360.579375, p75=360.579375, p95=360.579375, p98=360.579375, p99=360.579375, p999=360.579375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031061614020123102, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=759.624459, max=759.624459, mean=759.624459, stddev=0.0, p50=759.624459, p75=759.624459, p95=759.624459, p98=759.624459, p99=759.624459, p999=759.624459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031061192720059357, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=169.259042, max=169.259042, mean=169.259042, stddev=0.0, p50=169.259042, p75=169.259042, p95=169.259042, p98=169.259042, p99=169.259042, p999=169.259042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03106096715971506, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=217.7195, max=217.7195, mean=217.7195, stddev=0.0, p50=217.7195, p75=217.7195, p95=217.7195, p98=217.7195, p99=217.7195, p999=217.7195, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031060744497920734, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:12 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:47:12 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:47:12 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:12 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:12 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 14:47:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 14:47:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:47:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:47:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 14:47:13 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 14:47:13 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client +18-08-2023 14:47:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 14:47:13 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:13 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 14:47:13 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 14:47:13 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 14:47:13 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 14:47:13 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 14:47:13 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 +18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} +18-08-2023 14:47:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 +18-08-2023 14:47:13 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4949091797102522534 doesn't exist +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4949091797102522534. +18-08-2023 14:47:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4949091797102522534 +18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4949091797102522534 +18-08-2023 14:47:13 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4949091797102522534, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_0","table":"kafka_connector_test_table_4949091797102522534"} +18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4949091797102522534, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534 +18-08-2023 14:47:13 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] +18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset:null +18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4949091797102522534, topicPartition: test-0 +18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created +18-08-2023 14:47:13 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 +18-08-2023 14:47:13 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. +18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0,currentBufferSizeInBytes:211, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} +18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1} +18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 211 bytes, offset 1 - 1 +18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 14:47:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1040 +18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 +18-08-2023 14:47:14 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@3e6d4889[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] +18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec] +18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 +18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, idx=0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, enableParquetMemoryOptimization=false +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start +18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 105, 110, 116, 49, 54, 34, 58, 49, 50, 125] +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(12 bytes) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 16 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 16 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 40 to byteArray of 40 bytes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 40 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 63 to byteArray of 63 bytes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 168: end column +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 63B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 22B raw, 40B comp, 1 pages, encodings: [PLAIN] +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 168: end block +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 168: column indexes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 330: offset indexes +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 352: bloom filters +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 352: end +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1088: footer length = 736 +18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 736 => 224 2 0 0 +18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534, rowCount=1, startOffset=0, estimatedUncompressedSize=74.5, paddedChunkLength=1096, encryptedCompressedSize=1104, bdecVersion=THREE +18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, size=1104 +18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, size=1104, timeInMillis=233 +18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, idx=0 +18-08-2023 14:47:14 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 +18-08-2023 14:47:14 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 14:47:14 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec","md5":"5679568e03cc27abebdbafacebb985c5","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","chunk_start_offset":0,"chunk_length":1096,"chunk_length_uncompressed":74,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"4e6ca2c9f706aa30ac9b711e1d22e0e7","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370970226,"first_insert_time_in_ms":1692395233832,"last_insert_time_in_ms":1692395233832,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692395234255,"build_duration_ms":28,"upload_duration_ms":232},"bdec_version":3}],"role":"testrole_kafka","request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_2"} +18-08-2023 14:47:14 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 +18-08-2023 14:47:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 +18-08-2023 14:47:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 +18-08-2023 14:47:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 +18-08-2023 14:47:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:18 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 +18-08-2023 14:47:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 14:47:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 14:47:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 +18-08-2023 14:47:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 14:47:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:23 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 +18-08-2023 14:47:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 14:47:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 14:47:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 14:47:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 14:47:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:28 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 14:47:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 14:47:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 +18-08-2023 14:47:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 +18-08-2023 14:47:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 14:47:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:33 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 +18-08-2023 14:47:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 +18-08-2023 14:47:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 +18-08-2023 14:47:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 +18-08-2023 14:47:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 +18-08-2023 14:47:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:38 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 14:47:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 +18-08-2023 14:47:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 +18-08-2023 14:47:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 +18-08-2023 14:47:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:43 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 14:47:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 +18-08-2023 14:47:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 +18-08-2023 14:47:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:44 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 +18-08-2023 14:47:44 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:47:44 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset:1 +18-08-2023 14:47:44 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, channel sequencer=0, row sequencer=1 +18-08-2023 14:47:44 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} +18-08-2023 14:47:44 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 +18-08-2023 14:47:44 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 14:47:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=449, currentDiffMillis=453 +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1104, max=1104, mean=1104.0, stddev=0.0, p50=1104.0, p75=1104.0, p95=1104.0, p98=1104.0, p99=1104.0, p999=1104.0 +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=4, mean=0.09143120584939526, stddev=0.5150330101608892, p50=0.0, p75=0.0, p95=0.0, p98=2.0, p99=4.0, p999=4.0 +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=74, m1_rate=9.75676132696657, m5_rate=13.616657336513985, m15_rate=14.394546261321954, mean_rate=2.335870957716532, rate_unit=events/second +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1104, m1_rate=145.560331148258, m5_rate=203.14580675015458, m15_rate=214.7510685472897, mean_rate=34.84848828603301, rate_unit=events/second +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=28.986625, max=28.986625, mean=28.986625, stddev=0.0, p50=28.986625, p75=28.986625, p95=28.986625, p98=28.986625, p99=28.986625, p999=28.986625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031564586104728655, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=408.814917, max=408.814917, mean=408.814917, stddev=0.0, p50=408.814917, p75=408.814917, p95=408.814917, p98=408.814917, p99=408.814917, p999=408.814917, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031561944450599025, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=137.507583, max=137.507583, mean=137.507583, stddev=0.0, p50=137.507583, p75=137.507583, p95=137.507583, p98=137.507583, p99=137.507583, p999=137.507583, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0315616024409613, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=232.043417, max=232.043417, mean=232.043417, stddev=0.0, p50=232.043417, p75=232.043417, p95=232.043417, p98=232.043417, p99=232.043417, p999=232.043417, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03156135253663886, rate_unit=events/second, duration_unit=milliseconds +18-08-2023 14:47:44 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:44 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 14:47:44 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 +18-08-2023 14:47:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 14:47:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] + +18-08-2023 15:35:00 main DEBUG CodecFactory:71 - Snappy was not available +java.lang.NoClassDefFoundError: org/xerial/snappy/Snappy + at org.apache.avro.file.SnappyCodec$Option.(SnappyCodec.java:35) + at org.apache.avro.file.CodecFactory.snappyCodec(CodecFactory.java:69) + at org.apache.avro.file.CodecFactory.(CodecFactory.java:140) + at org.apache.avro.file.DataFileStream.resolveCodec(DataFileStream.java:160) + at org.apache.avro.file.DataFileStream.initialize(DataFileStream.java:144) + at org.apache.avro.file.DataFileReader.(DataFileReader.java:143) + at org.apache.avro.file.DataFileReader.(DataFileReader.java:130) + at com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry.toConnectData(SnowflakeAvroConverterWithoutSchemaRegistry.java:49) + at com.snowflake.kafka.connector.records.ProcessRecordTest.getAvroWithoutRegistryValue(ProcessRecordTest.java:184) + at com.snowflake.kafka.connector.records.ProcessRecordTest.data(ProcessRecordTest.java:69) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.runners.Parameterized$RunnersFactory.allParameters(Parameterized.java:424) + at org.junit.runners.Parameterized$RunnersFactory.(Parameterized.java:375) + at org.junit.runners.Parameterized$RunnersFactory.(Parameterized.java:360) + at org.junit.runners.Parameterized.(Parameterized.java:303) + at java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) + at java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) + at java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:490) + at org.junit.internal.builders.AnnotatedBuilder.buildRunner(AnnotatedBuilder.java:104) + at org.junit.internal.builders.AnnotatedBuilder.runnerForClass(AnnotatedBuilder.java:86) + at org.junit.runners.model.RunnerBuilder.safeRunnerForClass(RunnerBuilder.java:70) + at org.junit.internal.builders.AllDefaultPossibilitiesBuilder.runnerForClass(AllDefaultPossibilitiesBuilder.java:37) + at org.junit.runners.model.RunnerBuilder.safeRunnerForClass(RunnerBuilder.java:70) + at org.junit.runners.model.RunnerBuilder.runners(RunnerBuilder.java:125) + at org.junit.runners.model.RunnerBuilder.runners(RunnerBuilder.java:111) + at com.intellij.junit4.JUnit46ClassesRequestBuilder.collectWrappedRunners(JUnit46ClassesRequestBuilder.java:77) + at com.intellij.junit4.JUnit46ClassesRequestBuilder.getClassesRequest(JUnit46ClassesRequestBuilder.java:39) + at com.intellij.junit4.JUnit4TestRunnerUtil.buildRequest(JUnit4TestRunnerUtil.java:71) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:47) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.lang.ClassNotFoundException: org.xerial.snappy.Snappy + at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:581) + at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) + at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522) + ... 41 more +18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] invalid JDBC_LOG_DIR /dummy_dir_not_exist defaulting to /var/folders/hg/t5ql_h_514n9p951h854x1nw0000gn/T/ +18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] jdbc tracing directory = /usr +18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] jdbc tracing directory = /var/folders/hg/t5ql_h_514n9p951h854x1nw0000gn/T/ +18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: adsadas +18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] table name @123 should have at least 2 characters, start with _a-zA-Z, and only contains _$a-zA-z0-9 +18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] Current Snowflake Kafka Connector Version: 2.0.0 +18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: .*_cat, big_.*_cat +18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: .*_cat, .*_dog.*_cat +18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: 12321 +18-08-2023 15:35:00 main ERROR SnowflakeSinkTask_STATIC:94 - [SF_KAFKA_CONNECTOR] Invalid Input, Topic2Table Map disabled +18-08-2023 15:35:00 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:00 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:01 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:01 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:02 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:03 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:860, currentBufferedRecordCount:4, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=800, bufferRecordCountThreshold=10000000} +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=4, bufferSizeBytes=860, firstOffset=0, lastOffset=3} +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 4 records, 860 bytes, offset 0 - 3 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=4, bufferSizeBytes=860, firstOffset=0, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398108033, previousFlushTimeMs:1692398103019, bufferThresholdSeconds:5 +18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=215, firstOffset=4, lastOffset=4} +18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 215 bytes, offset 4 - 4 +18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=215, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Skip adding offset:0 to buffer for channel:TEST_0 because offsetPersistedInSnowflake:0, processedOffset:0 +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398113072, previousFlushTimeMs:1692398108058, bufferThresholdSeconds:5 +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=430, firstOffset=1, lastOffset=2} +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 430 bytes, offset 1 - 2 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=430, firstOffset=1, lastOffset=2}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:13 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:11878, currentBufferedRecordCount:2, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=10000000} +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=11878, firstOffset=0, lastOffset=1} +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 11878 bytes, offset 0 - 1 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=11878, firstOffset=0, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:1 +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398118160, previousFlushTimeMs:1692398113148, bufferThresholdSeconds:5 +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=5939, firstOffset=2, lastOffset=2} +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 5939 bytes, offset 2 - 2 +18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=5939, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:2 +18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:invalidNo +18-08-2023 15:35:18 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] The offsetToken string does not contain a parsable long:invalidNo for channel:TEST_0 +18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:19 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:19 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:0, reset kafka offset to:1 +18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:21 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:21 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Insert Row Error message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming Channel name:TEST_0 msg:Interrupted Exception +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:185, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=185, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 185 bytes, offset 0 - 0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=185, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 1 - 1 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 2 - 2 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 3 - 3 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 4 - 4 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Got the desired offset:1 from Kafka, we can add this offset to buffer for channel:TEST_0 +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 1 - 1 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 2 - 2 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 3 - 3 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 4 - 4 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:4 +18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:23 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:23 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:807, currentBufferedRecordCount:3, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=800, bufferRecordCountThreshold=10000000} +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=3, bufferSizeBytes=807, firstOffset=0, lastOffset=2} +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 3 records, 807 bytes, offset 0 - 2 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=3, bufferSizeBytes=807, firstOffset=0, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398129353, previousFlushTimeMs:1692398124341, bufferThresholdSeconds:5 +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=538, firstOffset=3, lastOffset=4} +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 538 bytes, offset 3 - 4 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=538, firstOffset=3, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:29 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:29 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:29 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:29 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:12098, currentBufferedRecordCount:2, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=10000000} +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=12098, firstOffset=0, lastOffset=1} +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 12098 bytes, offset 0 - 1 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=12098, firstOffset=0, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:1 +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398134389, previousFlushTimeMs:1692398129373, bufferThresholdSeconds:5 +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=6049, firstOffset=2, lastOffset=2} +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 6049 bytes, offset 2 - 2 +18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=6049, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:2 +18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:invalidNo +18-08-2023 15:35:34 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] The offsetToken string does not contain a parsable long:invalidNo for channel:TEST_0 +18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:35 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:0, reset kafka offset to:1 +18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 +18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:37 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:37 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Insert Row Error message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming Channel name:TEST_0 msg:Interrupted Exception +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:227, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=227, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 227 bytes, offset 0 - 0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=227, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 1 - 1 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 2 - 2 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 3 - 3 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 4 - 4 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 +18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Got the desired offset:1 from Kafka, we can add this offset to buffer for channel:TEST_0 +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 1 - 1 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 2 - 2 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 3 - 3 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 4 - 4 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false +18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 +18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:4 +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time is 0, it should be greater than 1' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: $@#$#@%^$12312 +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.topic2table.map' is invalid. Error message: 'Invalid snowflake.topic2table.map config format: $@#$#@%^$12312' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'key.converter' is invalid. Error message: 'Config:key.converter has provided value:com.snowflake.kafka.connector.records.SnowflakeJsonConverter. If ingestionMethod is:snowpipe_streaming, Snowflake Custom Converters are not allowed. +Invalid Converters:[com.snowflake.kafka.connector.records.SnowflakeJsonConverter, com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry, com.snowflake.kafka.connector.records.SnowflakeAvroConverter]' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' +Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' +Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' +Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' +Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' +Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.enable.schematization' is invalid. Error message: 'Schematization is only available with snowpipe_streaming.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid for configuration errors.tolerance: String must be one of: none, all' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.role.name' is invalid. Error message: 'Config:snowflake.role.name should be present if ingestionMethod is:snowpipe_streaming' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'provider' is invalid. Error message: 'Kafka provider config error:Unsupported provider name: Something_which_is_not_supported. Supported are: unknown,self_hosted,confluent' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records should be a positive integer. Provided:adssadsa' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'buffer.size.bytes is too low at 0. It must be 1 or greater.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes should be an integer. Provided:afdsa' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid_value for configuration snowflake.ingestion.method: String must be one of: snowpipe, snowpipe_streaming' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'value.converter' is invalid. Error message: 'Config:value.converter has provided value:com.snowflake.kafka.connector.records.SnowflakeJsonConverter. If ingestionMethod is:snowpipe_streaming, Snowflake Custom Converters are not allowed. +Invalid Converters:[com.snowflake.kafka.connector.records.SnowflakeJsonConverter, com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry, com.snowflake.kafka.connector.records.SnowflakeAvroConverter]' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jvm.proxy.port' is invalid. Error message: 'proxy host and port must be provided together' +Config value 'jvm.proxy.host' is invalid. Error message: 'proxy host and port must be provided together' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jvm.proxy.port' is invalid. Error message: 'proxy host and port must be provided together' +Config value 'jvm.proxy.host' is invalid. Error message: 'proxy host and port must be provided together' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid for configuration errors.log.enable: String must be one of: true, false' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time should be an integer. Invalid integer was provided:fdas' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'enable.streaming.client.optimization' is invalid. Error message: 'Streaming client optimization is only available with snowpipe_streaming.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records should be a positive integer. Provided:adssadsa' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is -1, it should at least 1' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'org.apache.kafka.connect.storage.StringConverter' is invalid. Error message: 'The value converter:org.apache.kafka.connect.storage.StringConverter is not supported with schematization.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time should be an integer. Invalid integer was provided:fdas' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'behavior.on.null.values' is invalid. Error message: 'Kafka config:behavior.on.null.values error:Invalid value invalid for configuration behavior.on.null.values: String must be one of: default, ignore' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is -1, it should at least 1' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] table name !@#@!#!@ should have at least 2 characters, start with _a-zA-Z, and only contains _$a-zA-z0-9 +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic name topic1 is duplicated +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: topic1, topic1 +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jmx' is invalid. Error message: 'Kafka config:jmx should either be true or false' + +18-08-2023 15:35:38 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] enable jvm proxy: 127.0.0.1:3128 and bypass proxy for hosts: *.snowflakecomputing.com|*.amazonaws.com +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.streaming.file.version' is invalid. Error message: 'snowflake.streaming.file.version is only available with ingestion type: snowpipe_streaming.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' +Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' +Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' +Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' +Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' +Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' +Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' +Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' +Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' +Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' +Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' +Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' +Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' +Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' + +18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time is 9, it should be greater than 10' + +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse JSON record +net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonParseException: Unexpected character ('a' (code 97)): Expected space separating root-level values + at [Source: (byte[])"123adsada"; line: 1, column: 5] +18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 100 + scrub.invalid.names = false + +18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse JSON record +net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonParseException: Unrecognized token 'fasfas': was expecting (JSON String, Number, Array, Object or token 'null', 'true' or 'false') + at [Source: (byte[])"fasfas"; line: 1, column: 7] +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: unknown bytes +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +null +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +null +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record +[SF_KAFKA_CONNECTOR] Exception: Invalid input record +Error Code: 0010 +Detail: Input record value can't be parsed +Message: Failed to parse AVRO record +Not an Avro data file. +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] the string provided for reader.schema is no valid Avro schema: com.fasterxml.jackson.core.JsonParseException: Unexpected character (':' (code 58)): expected a valid value (JSON String, Number, Array, Object or token 'null', 'true' or 'false') + at [Source: (String)"{"name":"test_avro","type":"record","fields":[{"name":"int","type":"int"},{"name":"newfield","type":"int","default": 1},{"name":"missingfield","type"::"int"}]}"; line: 1, column: 152] +18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. +18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. +18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record +null +18-08-2023 15:35:39 main DEBUG InternalUtils:70 - [SF_KAFKA_CONNECTOR] converted date: 2019-07-18T23:32:38Z +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://account.snowflake.com:80 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.snowflake.com:443 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: account.snowflake.com:80 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: account.snowflake.com +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://account.snowflake.com +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.snowflake.com +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.region.aws.privatelink.snowflake.com:443 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://org-account.snowflake.com:80 +18-08-2023 15:35:39 main INFO SnowflakeURL:46 - [SF_KAFKA_CONNECTOR] enabling JDBC tracing +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.region.aws.privatelink.snowflake.com:443 +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://org-account.privatelink.snowflake.com:80 +18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/test_topic/123/456_789_1692398139152.json.gz +18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/test_topic/123/456_key_1692398139158.gz +18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/test_topic/123/456_value_1692398139158.gz +18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:pipe, existing:[] +18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe","start_time":1692398139249,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start +18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe_streaming","start_time":1692398139255,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start +18-08-2023 15:35:39 main DEBUG KCLogger:23 - Setting MDC context enablement to: true. MDC context is only available for Apache Kafka versions after 2.3.0 +18-08-2023 15:35:39 main DEBUG KCLogger:23 - Setting MDC context enablement to: false. MDC context is only available for Apache Kafka versions after 2.3.0 +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 15:35:39 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming client, Exception message: cant close client, cause: missing exception cause +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:TEST_CONNECTOR +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Config is overridden for snowflake.streaming.file.version +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 15:35:39 main INFO Utils:22 - [SF_INGEST] Snowflake role is not provided, the default user role will be applied. +18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 15:35:39 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 +18-08-2023 15:35:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 15:35:39 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 15:35:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 15:35:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 15:35:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":null} +18-08-2023 15:35:39 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:39 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 0][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:39 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:39 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-0: Shutdown connection +18-08-2023 15:35:39 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 0][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:39 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 +18-08-2023 15:35:39 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:39 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:39 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:39 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 1][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:39 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:39 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:39 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-1: Shutdown connection +18-08-2023 15:35:39 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 1][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:39 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 +18-08-2023 15:35:40 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:40 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:40 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:40 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:40 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 2][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:40 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:40 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:40 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-2: Shutdown connection +18-08-2023 15:35:40 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 2][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:40 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 +18-08-2023 15:35:42 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:42 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:42 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:42 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:42 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 3][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:42 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:42 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:42 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-3: Shutdown connection +18-08-2023 15:35:42 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 3][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:42 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 +18-08-2023 15:35:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:35:44 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:35:44 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:35:46 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:46 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:46 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:46 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:46 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 4][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:46 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:46 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:46 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-4: Shutdown connection +18-08-2023 15:35:46 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 4][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:46 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 +18-08-2023 15:35:49 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:35:49 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:35:49 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:35:50 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:50 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:50 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:50 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:50 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 5][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:50 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:50 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:50 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-5: Shutdown connection +18-08-2023 15:35:50 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 5][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:50 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 +18-08-2023 15:35:54 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:35:54 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:35:54 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:35:54 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:54 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:54 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:54 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:54 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 6][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:54 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:54 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:54 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-6: Shutdown connection +18-08-2023 15:35:54 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 6][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:54 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 +18-08-2023 15:35:58 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:35:58 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:35:58 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:58 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:35:58 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 7][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:35:58 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:35:58 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:35:58 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-7: Shutdown connection +18-08-2023 15:35:58 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 7][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:35:58 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 +18-08-2023 15:35:59 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:35:59 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:35:59 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:02 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:02 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:02 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:02 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:02 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 8][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:02 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:02 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:02 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-8: Shutdown connection +18-08-2023 15:36:02 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 8][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:02 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 +18-08-2023 15:36:04 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:04 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:04 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:06 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:06 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:06 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:06 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:06 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 9][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:06 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:06 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:06 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-9: Shutdown connection +18-08-2023 15:36:06 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 9][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:06 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 +18-08-2023 15:36:09 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:09 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:09 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:10 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:10 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:10 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 10][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-10: Shutdown connection +18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 10][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ +18-08-2023 15:36:10 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 15:36:10 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient +18-08-2023 15:36:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:10 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:10 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:10 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 15:36:10 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 Thread-1 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 15:36:10 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 15:36:10 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 15:36:10 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 15:36:10 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 15:36:10 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 11][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-11: Shutdown connection +18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 11][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 +18-08-2023 15:36:10 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:10 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:10 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 12][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-12: Shutdown connection +18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 12][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:10 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 +18-08-2023 15:36:11 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:11 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:11 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:11 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:11 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 13][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:11 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:11 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:11 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-13: Shutdown connection +18-08-2023 15:36:11 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 13][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:11 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 +18-08-2023 15:36:13 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:13 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:13 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:13 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:13 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 14][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:13 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:13 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:13 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-14: Shutdown connection +18-08-2023 15:36:13 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 14][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:13 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 +18-08-2023 15:36:15 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:15 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:15 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:17 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:17 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:17 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:17 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:17 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 15][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:17 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:17 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:17 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-15: Shutdown connection +18-08-2023 15:36:17 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 15][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:17 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 +18-08-2023 15:36:20 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:20 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:20 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:21 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:21 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:21 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:21 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:21 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 16][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:21 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:21 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:21 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-16: Shutdown connection +18-08-2023 15:36:21 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 16][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:21 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 +18-08-2023 15:36:25 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:25 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:25 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:25 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:25 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:25 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:25 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:25 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 17][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:25 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:25 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:25 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-17: Shutdown connection +18-08-2023 15:36:25 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 17][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:25 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 +18-08-2023 15:36:29 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:29 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:29 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:29 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:29 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 18][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:29 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:29 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:29 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-18: Shutdown connection +18-08-2023 15:36:29 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 18][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:29 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 +18-08-2023 15:36:30 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:30 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:30 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:33 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:33 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:33 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:33 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:33 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 19][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:33 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:33 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:33 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-19: Shutdown connection +18-08-2023 15:36:33 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 19][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:33 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 +18-08-2023 15:36:35 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:35 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:35 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:37 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:37 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:37 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:37 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:37 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 20][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:37 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:37 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:37 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-20: Shutdown connection +18-08-2023 15:36:37 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 20][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:37 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 +18-08-2023 15:36:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:36:40 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:40 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:40 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:41 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:41 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 58 more +18-08-2023 15:36:41 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 21][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-21: Shutdown connection +18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 21][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ +18-08-2023 15:36:41 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 15:36:41 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient +18-08-2023 15:36:41 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:41 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:41 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:36:41 main ERROR StreamingClientProvider:94 - [SF_KAFKA_CONNECTOR] Current streaming client is invalid, recreating client +18-08-2023 15:36:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 15:36:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 15:36:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 15:36:41 main INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 15:36:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 15:36:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 15:36:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 22][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-22: Shutdown connection +18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 22][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 +18-08-2023 15:36:41 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:41 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:41 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 23][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-23: Shutdown connection +18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 23][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:41 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 +18-08-2023 15:36:42 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:42 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:42 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:42 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:42 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 24][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:42 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:42 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:42 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-24: Shutdown connection +18-08-2023 15:36:42 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 24][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:42 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 +18-08-2023 15:36:44 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:44 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:44 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:44 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:44 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 25][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:44 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:44 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:44 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-25: Shutdown connection +18-08-2023 15:36:44 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 25][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:44 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 +18-08-2023 15:36:46 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:46 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:46 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:48 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:48 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:48 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:48 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:48 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 26][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:48 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:48 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:48 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-26: Shutdown connection +18-08-2023 15:36:48 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 26][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:48 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 +18-08-2023 15:36:51 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:51 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:51 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:52 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:52 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:52 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:52 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:52 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 27][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:52 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:52 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:52 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-27: Shutdown connection +18-08-2023 15:36:52 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 27][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:52 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 +18-08-2023 15:36:56 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:36:56 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:36:56 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:36:56 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:36:56 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:36:56 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:56 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:36:56 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 28][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:36:56 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:36:56 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:36:56 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-28: Shutdown connection +18-08-2023 15:36:56 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 28][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:36:56 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 +18-08-2023 15:37:00 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:00 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:37:00 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:00 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:00 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 29][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:00 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:00 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:00 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-29: Shutdown connection +18-08-2023 15:37:00 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 29][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:00 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 +18-08-2023 15:37:01 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:01 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:01 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:04 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:04 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:37:04 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:04 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:04 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 30][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:04 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:04 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:04 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-30: Shutdown connection +18-08-2023 15:37:04 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 30][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:04 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 +18-08-2023 15:37:06 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:06 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:06 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:08 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:08 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:37:08 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:08 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:08 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 31][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:08 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:08 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:08 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-31: Shutdown connection +18-08-2023 15:37:08 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 31][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:08 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 +18-08-2023 15:37:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:11 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:11 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:12 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:12 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.base/java.lang.reflect.Method.invoke(Method.java:566) + at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) + at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) + at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) + at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) + at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) + at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) + at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) + at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runners.Suite.runChild(Suite.java:128) + at org.junit.runners.Suite.runChild(Suite.java:27) + at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) + at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) + at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) + at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) + at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) + at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) + at org.junit.runners.ParentRunner.run(ParentRunner.java:413) + at org.junit.runner.JUnitCore.run(JUnitCore.java:137) + at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) + at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) + at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) + at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) + at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) + at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 82 more +18-08-2023 15:37:12 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 main DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:12 main DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 32][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:12 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:12 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-32: Shutdown connection +18-08-2023 15:37:12 main DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 32][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ +18-08-2023 15:37:12 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 15:37:12 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient +18-08-2023 15:37:12 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:12 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:12 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:12 pool-11-thread-1 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-2 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-4 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-3 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-1 ERROR StreamingClientProvider:94 - [SF_KAFKA_CONNECTOR] Current streaming client is invalid, recreating client +18-08-2023 15:37:12 pool-11-thread-6 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-5 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-7 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-8 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-10 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-9 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid +18-08-2023 15:37:12 pool-11-thread-1 INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... +18-08-2023 15:37:12 pool-11-thread-1 DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... +18-08-2023 15:37:12 Thread-3 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread +18-08-2023 15:37:12 pool-11-thread-1 INFO SecurityManager:190 - Successfully created new JWT +18-08-2023 15:37:12 pool-11-thread-1 INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null +18-08-2023 15:37:12 pool-11-thread-1 INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 33][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-33: Shutdown connection +18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 33][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 +18-08-2023 15:37:12 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:12 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 34][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-34: Shutdown connection +18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 34][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:12 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 +18-08-2023 15:37:13 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:13 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:13 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:13 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:13 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 35][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:13 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:13 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:13 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-35: Shutdown connection +18-08-2023 15:37:13 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 35][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:13 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 +18-08-2023 15:37:15 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:15 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:15 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:15 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:15 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 36][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:15 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:15 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:15 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-36: Shutdown connection +18-08-2023 15:37:15 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 36][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:15 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 +18-08-2023 15:37:17 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:17 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:17 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:19 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:19 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:19 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:19 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:19 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 37][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:19 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:19 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:19 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-37: Shutdown connection +18-08-2023 15:37:19 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 37][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:19 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 +18-08-2023 15:37:22 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:22 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:22 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:23 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:23 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:23 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:23 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:23 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 38][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:23 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:23 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:23 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-38: Shutdown connection +18-08-2023 15:37:23 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 38][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:23 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 +18-08-2023 15:37:27 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:27 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:27 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:27 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:27 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:27 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:27 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:27 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 39][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:27 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:27 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:27 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-39: Shutdown connection +18-08-2023 15:37:27 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 39][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:27 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 +18-08-2023 15:37:31 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:31 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:31 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:31 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:31 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 40][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:31 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:31 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:31 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-40: Shutdown connection +18-08-2023 15:37:31 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 40][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:31 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 +18-08-2023 15:37:32 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:32 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:32 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:35 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:35 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:35 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:35 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:35 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 41][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:35 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:35 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:35 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-41: Shutdown connection +18-08-2023 15:37:35 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 41][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:35 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 +18-08-2023 15:37:37 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:37 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:37 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:37:39 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:39 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:39 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:39 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:39 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 42][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:39 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:39 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:39 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-42: Shutdown connection +18-08-2023 15:37:39 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 42][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:39 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 +18-08-2023 15:37:42 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:42 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:42 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:37:43 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +18-08-2023 15:37:43 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) +net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) + at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) + at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) + at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) + at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) + at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) + at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) + at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) + at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) + at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) + at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) + at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) + at org.mockito.Answers.answer(Answers.java:100) + at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) + at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) + at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) + at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) + at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) + at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) + at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) + at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) + at java.base/java.lang.Thread.run(Thread.java:829) +Caused by: java.net.ConnectException: Connection refused (Connection refused) + at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) + at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) + at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) + at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) + at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) + at java.base/java.net.Socket.connect(Socket.java:609) + at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) + at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) + ... 40 more +18-08-2023 15:37:43 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:43 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default +18-08-2023 15:37:43 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context +18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 43][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] +18-08-2023 15:37:43 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:37:43 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 +18-08-2023 15:37:43 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-43: Shutdown connection +18-08-2023 15:37:43 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded +18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 43][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] +18-08-2023 15:37:43 pool-11-thread-1 INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ +18-08-2023 15:37:43 pool-11-thread-1 DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread +18-08-2023 15:37:43 pool-11-thread-1 ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient +18-08-2023 15:37:43 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: +Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] +Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] + +18-08-2023 15:37:43 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:37:43 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS +18-08-2023 15:38:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:38:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:39:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:39:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:40:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:40:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:41:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:41:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:42:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:42:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:43:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:43:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:44:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:44:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:45:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections +18-08-2023 15:45:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS +18-08-2023 15:46:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 +18-08-2023 15:46:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1978826340154525689_0 +18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1978826340154525689 +18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 +18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 dropped +18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1978826340154525689_0 dropped +18-08-2023 15:46:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 +18-08-2023 15:46:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6792973406562441213_0 +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6792973406562441213 +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 dropped +18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6792973406562441213_0 dropped +18-08-2023 15:46:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 +18-08-2023 15:46:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1110551375533109079_0 +18-08-2023 15:46:38 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:38 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:38 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:38 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1110551375533109079 +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 dropped +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1110551375533109079_0 dropped +18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 +18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6012288537030407474_0 +18-08-2023 15:46:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6012288537030407474 +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 dropped +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6012288537030407474_0 dropped +18-08-2023 15:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 +18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3110947853332713949_0 +18-08-2023 15:46:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:39 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3110947853332713949 +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 dropped +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3110947853332713949_0 dropped +18-08-2023 15:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 +18-08-2023 15:46:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4233424965819834485_0 +18-08-2023 15:46:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:40 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4233424965819834485 +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 dropped +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4233424965819834485_0 dropped +18-08-2023 15:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 +18-08-2023 15:46:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3313146833192302265_0 +18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3313146833192302265 +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 dropped +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3313146833192302265_0 dropped +18-08-2023 15:46:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 +18-08-2023 15:46:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2332824327087748017_0 +18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2332824327087748017 +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 dropped +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2332824327087748017_0 dropped +18-08-2023 15:46:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 +18-08-2023 15:46:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4326952369058869337_0 +18-08-2023 15:46:43 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:43 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:43 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:43 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4326952369058869337 +18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 dropped +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4326952369058869337_0 dropped +18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 +18-08-2023 15:46:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8264379696022517270_0 +18-08-2023 15:46:44 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:44 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:44 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:44 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8264379696022517270 +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 dropped +18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8264379696022517270_0 dropped +18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:45 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 +18-08-2023 15:46:45 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8070171452629621824_0 +18-08-2023 15:46:45 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8070171452629621824 +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 dropped +18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8070171452629621824_0 dropped +18-08-2023 15:46:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:46 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 +18-08-2023 15:46:46 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7092255300802014642_0 +18-08-2023 15:46:46 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7092255300802014642 +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 dropped +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7092255300802014642_0 dropped +18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:47 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:47 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:46:47 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:48 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:46:48 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 +18-08-2023 15:49:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4349441731007330241_0 +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4349441731007330241 +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 dropped +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4349441731007330241_0 dropped +18-08-2023 15:49:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:20 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 +18-08-2023 15:49:20 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5888119348378217666_0 +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5888119348378217666 +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 dropped +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5888119348378217666_0 dropped +18-08-2023 15:49:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 +18-08-2023 15:49:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6617430036033989156_0 +18-08-2023 15:49:21 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:21 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:21 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:22 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6617430036033989156 +18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 +18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 dropped +18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6617430036033989156_0 dropped +18-08-2023 15:49:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 +18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4976427781984260095_0 +18-08-2023 15:49:23 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4976427781984260095 +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 dropped +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4976427781984260095_0 dropped +18-08-2023 15:49:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 +18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4606067486270226287_0 +18-08-2023 15:49:23 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:23 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4606067486270226287 +18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 +18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 dropped +18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4606067486270226287_0 dropped +18-08-2023 15:49:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 +18-08-2023 15:49:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5267653630133644600_0 +18-08-2023 15:49:25 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:25 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:25 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:25 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5267653630133644600 +18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 +18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 dropped +18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5267653630133644600_0 dropped +18-08-2023 15:49:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 +18-08-2023 15:49:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4965523502332819922_0 +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4965523502332819922 +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 dropped +18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4965523502332819922_0 dropped +18-08-2023 15:49:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 +18-08-2023 15:49:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1463553278395633792_0 +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1463553278395633792 +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 dropped +18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1463553278395633792_0 dropped +18-08-2023 15:49:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 +18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4038668653217555244_0 +18-08-2023 15:49:28 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4038668653217555244 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 dropped +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4038668653217555244_0 dropped +18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 +18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6612709555054720227_0 +18-08-2023 15:49:28 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:28 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6612709555054720227 +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 dropped +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6612709555054720227_0 dropped +18-08-2023 15:49:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 +18-08-2023 15:49:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3762733667586168679_0 +18-08-2023 15:49:29 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3762733667586168679 +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 dropped +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3762733667586168679_0 dropped +18-08-2023 15:49:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 +18-08-2023 15:49:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5874843966951528984_0 +18-08-2023 15:49:30 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5874843966951528984 +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 dropped +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5874843966951528984_0 dropped +18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:31 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:32 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:32 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:32 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:32 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:33 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:33 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:33 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:33 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:49:33 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:33 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:34 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. +18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:49:34 main INFO StringConverterConfig:376 - StringConverterConfig values: + converter.encoding = UTF-8 + converter.type = value + +18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 +18-08-2023 15:51:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698054729379700747_0 +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5698054729379700747 +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 dropped +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698054729379700747_0 dropped +18-08-2023 15:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 +18-08-2023 15:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8992954341468273479_0 +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8992954341468273479 +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 dropped +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8992954341468273479_0 dropped +18-08-2023 15:51:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 +18-08-2023 15:51:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1848613164558782106_0 +18-08-2023 15:51:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:10 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1848613164558782106 +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 dropped +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1848613164558782106_0 dropped +18-08-2023 15:51:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 +18-08-2023 15:51:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9013078777524223289_0 +18-08-2023 15:51:11 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:11 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:11 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:11 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9013078777524223289 +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 dropped +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9013078777524223289_0 dropped +18-08-2023 15:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 +18-08-2023 15:51:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698617716216278091_0 +18-08-2023 15:51:12 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:12 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:12 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:12 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5698617716216278091 +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 dropped +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698617716216278091_0 dropped +18-08-2023 15:51:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 +18-08-2023 15:51:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_595050023951904600_0 +18-08-2023 15:51:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + specific.avro.reader = false + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:13 main INFO AvroDataConfig:376 - AvroDataConfig values: + connect.meta.data = true + discard.type.doc.default = false + enhanced.avro.schema.support = false + generalized.sum.type.support = false + schemas.cache.config = 1000 + scrub.invalid.names = false + +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_595050023951904600 +18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 dropped +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_595050023951904600_0 dropped +18-08-2023 15:51:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 +18-08-2023 15:51:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6259484133267789406_0 +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6259484133267789406 +18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 dropped +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6259484133267789406_0 dropped +18-08-2023 15:51:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 +18-08-2023 15:51:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6888502239215652181_0 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6888502239215652181 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 +18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 dropped +18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6888502239215652181_0 dropped +18-08-2023 15:51:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 +18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection +18-08-2023 15:51:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4542948488367988717 +18-08-2023 15:51:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4542948488367988717_0 +18-08-2023 15:51:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: + auto.register.schemas = true + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + +18-08-2023 15:51:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: + auto.register.schemas = true + avro.reflection.allow.null = false + avro.remove.java.properties = false + avro.use.logical.type.converters = false + basic.auth.credentials.source = URL + basic.auth.user.info = [hidden] + bearer.auth.credentials.source = STATIC_TOKEN + bearer.auth.token = [hidden] + context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy + id.compatibility.strict = true + key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + latest.compatibility.strict = true + max.schemas.per.subject = 1000 + normalize.schemas = false + proxy.host = + proxy.port = -1 + schema.reflection = false + schema.registry.basic.auth.user.info = [hidden] + schema.registry.ssl.cipher.suites = null + schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] + schema.registry.ssl.endpoint.identification.algorithm = https + schema.registry.ssl.engine.factory.class = null + schema.registry.ssl.key.password = null + schema.registry.ssl.keymanager.algorithm = SunX509 + schema.registry.ssl.keystore.certificate.chain = null + schema.registry.ssl.keystore.key = null + schema.registry.ssl.keystore.location = null + schema.registry.ssl.keystore.password = null + schema.registry.ssl.keystore.type = JKS + schema.registry.ssl.protocol = TLSv1.3 + schema.registry.ssl.provider = null + schema.registry.ssl.secure.random.implementation = null + schema.registry.ssl.trustmanager.algorithm = PKIX + schema.registry.ssl.truststore.certificates = null + schema.registry.ssl.truststore.location = null + schema.registry.ssl.truststore.password = null + schema.registry.ssl.truststore.type = JKS + schema.registry.url = [http://fake-url] + use.latest.version = false + use.schema.id = -1 + value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy + diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java index d7b86c39a..4adbc121f 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnectorConfig.java @@ -678,7 +678,7 @@ public static String[] names() { @Override public String toString() { - return name(); + return name().toLowerCase(Locale.ROOT); } } diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java index 3d27734e2..1831cec31 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java @@ -172,18 +172,6 @@ public void start(final Map parsedConfig) { final long bufferFlushTime = Long.parseLong(parsedConfig.get(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC)); - // Falling back to default behavior which is to ingest an empty json string if we get null - // value. (Tombstone record) - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; - if (parsedConfig.containsKey(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)) { - // we can always assume here that value passed in would be an allowed value, otherwise the - // connector would never start or reach the sink task stage - behavior = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( - parsedConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)); - } - // we would have already validated the config inside SFConnector start() boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; if (parsedConfig.containsKey(SnowflakeSinkConnectorConfig.JMX_OPT)) { diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index e733af34a..73dde0dfe 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -94,10 +94,9 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { // Setting the default value in constructor // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) this.behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( - connectorConfig.getOrDefault( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) + .filter(behavior -> behavior.toString().equalsIgnoreCase(connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + .findAny().orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); } /** diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 798a86304..ac22e0e54 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -18,6 +18,8 @@ import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import com.snowflake.kafka.connector.records.RecordService; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; + +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -122,11 +124,13 @@ public SnowflakeSinkServiceV2( Boolean.parseBoolean( connectorConfig.getOrDefault( SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + + // Setting the default value in constructor + // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) this.behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( - connectorConfig.getOrDefault( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) + .filter(behavior -> behavior.toString().equalsIgnoreCase(connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + .findAny().orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 04ac21006..505db2941 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -125,8 +125,8 @@ public void test() throws IOException { } @Test(expected = SnowflakeKafkaConnectorException.class) - public void testEmptyValue() { - RecordService service = new RecordService(); + public void testEmptyValueDisabledTombstone() { + RecordService service = new RecordService(false, false); SinkRecord record = new SinkRecord(topic, partition, null, null, Schema.STRING_SCHEMA, null, partition); @@ -134,10 +134,10 @@ public void testEmptyValue() { } @Test(expected = SnowflakeKafkaConnectorException.class) - public void testEmptyValueSchema() throws IOException { + public void testEmptyValueSchemaDisabledTombstone() throws IOException { JsonNode data = mapper.readTree("{\"name\":123}"); SnowflakeRecordContent content = new SnowflakeRecordContent(data); - RecordService service = new RecordService(); + RecordService service = new RecordService(false, false); SinkRecord record = new SinkRecord(topic, partition, null, null, null, content, partition); service.getProcessedRecordForSnowpipe(record); From cd097131b9b3650238ca588cbab684830d1b64f8 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Fri, 18 Aug 2023 16:16:12 -0700 Subject: [PATCH 08/42] autoformatting --- .../internal/SnowflakeSinkServiceV1.java | 11 +++++-- .../streaming/SnowflakeSinkServiceV2.java | 12 ++++++-- .../connector/records/RecordService.java | 3 +- .../internal/TombstoneRecordIngestionIT.java | 29 +++++++------------ 4 files changed, 31 insertions(+), 24 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index 73dde0dfe..2ffce054b 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -95,8 +95,15 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) this.behaviorOnNullValues = Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) - .filter(behavior -> behavior.toString().equalsIgnoreCase(connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) - .findAny().orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + .filter( + behavior -> + behavior + .toString() + .equalsIgnoreCase( + connectorConfig.get( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + .findAny() + .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); } /** diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index ac22e0e54..f91a117f4 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -18,7 +18,6 @@ import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import com.snowflake.kafka.connector.records.RecordService; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; - import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -129,8 +128,15 @@ public SnowflakeSinkServiceV2( // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) this.behaviorOnNullValues = Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) - .filter(behavior -> behavior.toString().equalsIgnoreCase(connectorConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) - .findAny().orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + .filter( + behavior -> + behavior + .toString() + .equalsIgnoreCase( + connectorConfig.get( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + .findAny() + .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 5b9c4489c..fd5eba993 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -88,7 +88,8 @@ public class RecordService { private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); private Map connectorConfig; private boolean enableSchematization; - private boolean ingestTombstoneRecords = true; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion + private boolean ingestTombstoneRecords = + true; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 48bfd510a..9883b8b05 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -6,12 +6,11 @@ import com.snowflake.kafka.connector.dlq.InMemoryKafkaRecordErrorReporter; import com.snowflake.kafka.connector.internal.streaming.InMemorySinkTaskContext; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; -import java.nio.charset.StandardCharsets; -import java.util.*; - import io.confluent.connect.avro.AvroConverter; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import java.nio.charset.StandardCharsets; +import java.util.*; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -20,7 +19,6 @@ import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.storage.Converter; import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -37,12 +35,12 @@ public static Collection input() { ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter }, { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter }, { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter }, { IngestionMethodConfig.SNOWPIPE, @@ -61,8 +59,8 @@ public static Collection input() { ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter }, { - IngestionMethodConfig.SNOWPIPE_STREAMING, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter }, { IngestionMethodConfig.SNOWPIPE_STREAMING, @@ -207,18 +205,13 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { // make normal record byte[] normalRecordData = "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8); if (isAvroConverter) { - SchemaBuilder schemaBuilder = - SchemaBuilder.struct() - .field("int16", Schema.INT16_SCHEMA); + SchemaBuilder schemaBuilder = SchemaBuilder.struct().field("int16", Schema.INT16_SCHEMA); - Struct original = - new Struct(schemaBuilder.build()) - .put("int16", (short) 12); + Struct original = new Struct(schemaBuilder.build()).put("int16", (short) 12); normalRecordData = converter.fromConnectData(topic, original.schema(), original); } - SchemaAndValue record2Input = - converter.toConnectData(topic, normalRecordData); + SchemaAndValue record2Input = converter.toConnectData(topic, normalRecordData); long record2Offset = 1; SinkRecord record2 = new SinkRecord( From 928b9439033a0bd80b46a7572e4d787e4ff39f25 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Fri, 18 Aug 2023 16:28:20 -0700 Subject: [PATCH 09/42] build passes locally --- .../connector/internal/streaming/TopicPartitionChannel.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java index 3821c7f27..91d6c0aca 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java @@ -200,7 +200,7 @@ public TopicPartitionChannel( kafkaRecordErrorReporter, sinkTaskContext, null, /* Null Connection */ - new RecordService(null /* Null Telemetry Service*/), + new RecordService(), null); } From 00bee044298f7213e60beba774f23f8da81fe11a Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 15:48:43 -0700 Subject: [PATCH 10/42] personal nits --- sf.log.1 | 79748 ---------------- .../kafka/connector/ConnectorConfigTest.java | 8 +- .../internal/TombstoneRecordIngestionIT.java | 253 - .../streaming/SnowflakeSinkServiceV2IT.java | 43 - 4 files changed, 4 insertions(+), 80048 deletions(-) delete mode 100644 sf.log.1 diff --git a/sf.log.1 b/sf.log.1 deleted file mode 100644 index 311889f12..000000000 --- a/sf.log.1 +++ /dev/null @@ -1,79748 +0,0 @@ -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc8597a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ef7c69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cef81ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d304308 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@724cd85d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48fafe49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5808cc24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141041,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@572fec2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@433558a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ca2fe3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d57063 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab3aa8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a3f13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48326a4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@218d5d42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@569d5c0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d236d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa651b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@140a6048 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28cb13cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45204bf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28e6a12e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fa99c5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f0cbf0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583408d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45463d28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ae2296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c203daf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64aacc7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10f102ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141042,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c28a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a8ca0a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f77696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@308b55e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56746279 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@298259be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad2c9de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509caf31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258e78f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e7a8a57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32ca2c7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71ae830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@518551d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b675187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78ad2f40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2826130d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d6cc1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@721c448e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63dab2a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bfc7baf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eafced6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e07c1b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38fe3d79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141043,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c718ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@407ebb7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c4b0c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39184e51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@210100f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1411cac1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4186021e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a17a6b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1338378b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@965a0ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7882d52f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fa786e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d7172f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9b7dd61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67725ad5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24211015 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e140db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@350ac933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40db89b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c568803 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b9eaca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719e43f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141044,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b935348 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ddd6a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68198ba2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e9a282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2161acc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca45edf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28449072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4554da4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@697f49a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4783a0fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b486f7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28ad6ad5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b8c5461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@615287be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69693767 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad1035d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a7753a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63d6387 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26cfae28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a69a47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce0314c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ac420c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac916ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141045,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8ddf43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aafd0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@445573d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7699fb4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@524a4518 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d898db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7c8087 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54864de4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5464b0e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12bce4fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4d3696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d42b176 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd0b5cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793e2bbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb54b88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63949289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ae32df9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@628890c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43407e1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@487ac20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7778d5ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141046,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51501d52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11a011c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28ff497d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@191b2c7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48f01887 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@594f0665 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b290890 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b91857a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36d88af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22b0da89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619e2bc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2663d8db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49241448 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d64fb77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7db4350f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45a98f62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cc655cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66b3bc67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af3e1f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3328ab50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ddec8f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@567509ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@732c86ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141047,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@528c1f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eea18f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@202c6ad0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a08bc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7444f388 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ae77057 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39deff31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a93bd65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5918d70c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18bc71b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d532e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@413ac5cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e05da57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@254f01d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a726da4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4df1ffbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dd64337 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@604aa4ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@291cfa0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd2ec8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6412c53c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@514a40a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141048,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f483c29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638235f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11cb279f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12072a46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cf0e55b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eaf54b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76106b23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dee3abd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@167fd666 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b13afb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278d1afd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b9ae763 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63bedc90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388354fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7716fbe1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bebbb9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e24a82e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c222fb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cab1626 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a6a696f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3114c1d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c4627e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38580c61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141049,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41802b14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568d5891 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b7944c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@233a60cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a22734 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd7a201 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e999249 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed5b756 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cbcbc3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e19d803 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c100c53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f6c6180 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16987f16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a17406 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a7f9c15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64386bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@537639aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0a6686 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b3d866 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@125fe195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7086311d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d4557dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b23defb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141050,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769dd743 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37d20e54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108e4ccc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15df8c20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d606617 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a677941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@486db8ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@213a934c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe419a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e58d06e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b57bd98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e02ed4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61a4d026 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1907a7dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f1a6219 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@468b9832 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e95525 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25f42b83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49928f8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53c2c486 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c9f3a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce56b0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141051,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d669e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47573aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5376e441 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@647c4033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1617bc27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3950756c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5de0daeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b9b74bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c60d6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a07fd0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@116b5124 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11d5e921 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51b82b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24a24bb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f8bdc74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a9bc45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4159ad91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7790b9ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4649a316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a1e640e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a76ff9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@240ae318 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fd0b59c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141052,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c79a7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b72e5fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1893d9af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3245e374 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@86b8b3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74cae2f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7292036f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd1000a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bb77491 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c99c3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a226d30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d94e32e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d870ff1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49fb5008 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20d9ffea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aeee54c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f338967 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cdbf599 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43127947 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3b20ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d60ecfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee4a7f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141053,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e6c212f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59a7484e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53aa4083 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8aa9e0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4700b36f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248666f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252994b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79af34e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b767b74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cd0734d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4535ee61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45bde23e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d189c26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627dc4d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@851681b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2257d73d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b2796fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27f09416 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24622143 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3890f4cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f171d5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c646c8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39393c7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141054,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b0910c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11da817b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ef58fc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4edfbe88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce56eae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9f1ff10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1be8fa61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f03c6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f87a7c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4908d85b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ef08e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cd57552 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e1e4c8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415b8f13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c9e7f91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b76f0b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b09944 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a8ebdb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45b33b81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49cc88da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab5421e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@253925f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@790a6de6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141055,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54ada045 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e8ad7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589200af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@725d3306 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b4b0a80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab9cdd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e62558 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6938fd01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a34f5ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e17303f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d181840 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63e62417 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e223452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9037d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4be19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56c0e408 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79efda26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b15480f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a901702 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ba14afc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c989d44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0cfbe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141056,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@176ecf4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68981f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e235edf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@644a3d48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bf9a22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff3255b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b624fdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e778385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e060e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@116f5cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8eafde7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a033cd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e20b20f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2541a5e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a1b716c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@696d1767 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2296608f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a509fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2be10f99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a36e3ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b357c7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3596395b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16a865b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141057,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b9ed744 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd88cd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f003216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a50799c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3709a365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68682f67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@766620fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f332b37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@455ba53f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce1b9e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bc70556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d8bf98a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e1d759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74ae6298 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675034ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3545d627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0594c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29b88933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fa93fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54463c92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56e7590f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141058,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bfba3f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d29bf6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@586ea6f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ada3d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52af7a98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228d6565 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff92ec6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@569397f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fc15c05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36aeb8d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70673fd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a81321e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd5603f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c7e1100 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@452e66a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@760cbdd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123cffb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eb60ae1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589b2ab9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b6ae64e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ede79d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60618deb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61aebdbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141059,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d6a6f51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@303791ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cd54cf6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4802498d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@663cb73b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a09e3d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ccdb098 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d9ccad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45df3e22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606d5a31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11d57fd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@196165e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25f366c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc51c9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1554ed69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@708ec79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f36701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed59d69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd8107f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c734eb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e9ae359 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb60536 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141060,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30a1b937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66713f1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f4888a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@131ba9ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e4ed29f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a89195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5abe0c88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a5bbec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca759cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49064df5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63f6c211 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a357f58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c2a2ce2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b8a577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20b13db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7335737d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e050cc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c84836f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46cbdeef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ebdf3d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f8b7e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd46d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141061,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18fb4e2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69290d93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1493c0d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a919294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285923df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93df6d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a21a5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65412d88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7ad24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@628a400c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e9e6cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d28b815 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@249fb819 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a516a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ac45571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7274a82c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bf2890b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175eaaf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f391ab2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48dbe4b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@775e9670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141062,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15b5836c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@428ee7e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55676d6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56898ca4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f9edbfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a9a901c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34789a3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0edcbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37d6838b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c2ef1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499b3b63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11a8cdef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12e1de8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c7571cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c250502 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb562dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@673ce6ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70d00094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74591699 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24aa294c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2423a2e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5407d251 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141063,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68d6fb96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc34f89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24ced27a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eb119e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11e441e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@165ceb51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4453a6e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26b99d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31333b2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4044583a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19798266 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@470275c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f40cc44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@da7cc1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5818157e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c60dfc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@652b10e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56dfe762 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e74bb22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e0fbfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797167bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503c7b00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141064,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b7a678e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78067430 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ab88834 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@396f0618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c140ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2d1950 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@549acb09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15045844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e9bae68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c0ec972 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4e9f79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93f268c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac4312d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@504b3e5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab4d679 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6acf98e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513981c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5889ec7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6432ee09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@722c5ada -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f06a296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141065,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@787b511d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bcaaabd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b20db94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4168d874 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b99afd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d784b75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2099cb35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27351c2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f40443 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7728d5cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19067878 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@165df33e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea33baa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f143738 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@257b4b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c469888 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b1f86c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@551cd36d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a40714f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42aa3e57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4504814d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141066,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76b0b53b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd378ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c18bd80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d7753a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e9d225f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1acf0f07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b25563 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cfd6d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337793c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5122059b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ba5b91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a392641 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25df7178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6490a685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66f41f55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c36e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b96c38e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad3e2f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c1b4c4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18d4fe3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75b3c419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50524402 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@768d1373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141067,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49173692 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22efb0a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60df82ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed7034d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e0419a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51b1fa5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d67c834 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bec67be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258bae61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b36d384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@86392be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60dad59b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18da8747 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55bdcd2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415f5340 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779f20fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3361f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c372e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b22a136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35b4e667 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a81df6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74acad2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a299998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141068,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48bcdb64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d2e2828 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cfa135a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a89ae04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34c021ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c33012 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47f8893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5351ff96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@662a5a47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27a8b530 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f6b8a46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a6372 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3756f924 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b5cbcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568f5281 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de3ad1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c17cf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d511477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e64a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2619fa59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bb20ca9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e355704 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141069,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de090be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54426e69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f2ef70c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b57109a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220b162e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@657d0520 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d730814 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e0304e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24c3d368 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@291a86d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c4c084a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a2e3127 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304ceb99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d1dff6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61bfb0da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae312c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d243225 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fe07a4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73452e0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71b24faa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6beea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7135cd51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bda800a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141070,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48f645c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aaaa48f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@552abfce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a8fafb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ad0741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7c5f07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f2845e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@580d819a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60a212d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dcbfda1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4001e6fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7434be37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4703ede -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5684ed04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19fdc72e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e91720 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dea562d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf76e59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b441a5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b145401 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d03870 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d20115 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@382f77db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141071,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51729883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4342965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@472d4a5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff7cce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@410c4a52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64563c17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4904fd70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36e2e192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@159f73c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40fc53e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7dc9f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f3253d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@615d00bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@439c1c15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36e4ffb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1edf60dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@178ef365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34512935 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45b1a9d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@200de030 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40f83a8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5dab24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35ea7a6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141072,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a290080 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b846108 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b90f1d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d4fc7da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f50ebf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67044130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b760a4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78747ded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40c31e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50bda8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e14941f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65d40d25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dbc27fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66b8654d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c16b1e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ee266b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6531555b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4006c6bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@956bead -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c31c92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb7027c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58720a39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@646d744c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141073,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d13c3df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ba192e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@634a666c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c0ab15d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f249a38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68b53fb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58fc4dc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c59fd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@adb2c88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b52f2f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@717a7c57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8c46a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fe082d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337930a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11436a5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f24afb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24b9528e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ebfa2b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54241a88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1726db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50d42eb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@735d8c0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141074,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d938b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ff76c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71afd050 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8d37ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20df8e71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe4d991 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c6bf6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57f17228 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b81c2a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7250ba11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17884ee1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bb41e0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e254917 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39d2f2f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c784c6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d74ba6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63dec7ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d35ec43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6380daac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e373701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bc1d86a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@297462a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141075,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cc4889d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b2c213b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eebbebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6400988e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6864f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4687b9bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75aae18e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@158ed35d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a8e62e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@213f7a57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33b596f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@604c766e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49fc99fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134bbf54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a626c73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499e8c57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6400cff3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aff6d38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73279de0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ee64be7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cfb7a9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dbc5495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32bb0e5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141076,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4efed29c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a6765e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26219503 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c82be6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7450ff50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4e6302 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@546d6845 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dc5cd68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e9c19ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73c7235d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c575220 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fda784f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7197c7aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b80df0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4db05c0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71144df5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e716eea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b4be85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c2bc2c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f035556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab06e62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd6ad27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ef95ba4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141077,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5504c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32bfaabe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b39965b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f49d8e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c09f486 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23ac59e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64608883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@757fda2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5671bdc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17d385ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77173344 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c323930 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27f61cc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c830b7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@624080bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f56de7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18d1c35e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cbde3d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a1efba7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73024990 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65dfbf9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26bf6b8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b417f3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141078,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a6eeb0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d5808b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f6b7444 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f435c03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22f8f46c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce7f960 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@84a79ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d5c8fa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e247f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b1d8974 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@504314e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9bbb0d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43aca3de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241dc4f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c312093 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38d808ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eccee5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@543cb307 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@316c34ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490af414 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304d5d67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15874e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141079,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c1c10e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff52576 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1509ef8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b476ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8a204e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e9a5e13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b599f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@404080d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@585d3918 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@648b9ec6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c69c993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38eb80ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9cffb32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310a2963 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31d4b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671446dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11743766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d5451a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18fd51eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1ace87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d650d45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c37866 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141080,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c3155c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f2695b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aecb631 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2741ef42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf47703 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4def5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb6adfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a564b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb14f29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39e4c95c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63e77ae7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f0b1ac6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a56d409 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5674036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0f323f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a85ce72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@265fa5b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@190637ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dbc606f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@196ecff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c7a373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a54e776 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34355d4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141081,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4b2556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2468c977 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@694ae5db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770b22bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7568d89e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a98435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2973cf26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7431676 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34dc9228 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f8f30aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5832d882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47dd8554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17401255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc6a77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe45406 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b89f8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73fdfbf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1190b829 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c399a79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cdd06b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40e34cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5426e57c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64fc47d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141082,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a02568 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7c2d41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69922477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f562449 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e8cd7ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5571f31b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db8d45e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7392c9cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b7b51b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@714886e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab9b6bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f80d089 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e07c027 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b4d78c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c161d75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@156b8ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255d7094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7586cfc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29a789b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e22cd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef53980 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d4e3b67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7831e541 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141083,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9449273 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@efa93e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12731459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@651ff0bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e53161 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12dace1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1801d1c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2817051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630e3759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27e0bf0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31f9f5f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38731863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad554cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8efa674 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76c7acc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d7127a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56247767 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@238da7d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6867a6c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d2d71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a7fd90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d5a6f1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141084,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@685eb7fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7173925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b100f2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25dc9bed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38a009cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c0c1eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d62e746 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c201ef4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34f6a0ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b9f006 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ebb6900 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62601d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631000cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1852f56f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d946c0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ffa10d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c0f47ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@237936d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d03dab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32600360 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6404a001 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ea19da6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141085,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d4420ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f0d5f65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e48f22d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c061aef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505b0291 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35c88988 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c18cc86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b57888 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4872b585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3987e1ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43017680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289b3750 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7518615b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73174783 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bea8a17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aba1b8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c9eb938 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@357cc277 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa95cfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40555256 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@498b741d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab77821 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a69913e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141086,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6044b5c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72fd1e4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53aa1640 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@150c9d6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5023f881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab683a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49ad84a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383ba737 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7713b741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@946a9ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ceb8c8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490c9439 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23cf9be6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ff845f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f07356c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4207d9e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc86ae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40427373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16389242 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8cc9fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34cd1102 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c73503 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35bf113b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141087,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3970e550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e9b67d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd8f8ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a30067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48c73186 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b53cbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c56a7cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72982c18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a278cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3becd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637a4294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50b15a0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40452ccd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@558d0d70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64528bb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee9d410 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5723b444 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6af8c55d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a87e486 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6abeea92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20f5e215 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0b197b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141088,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290d6fa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34faea61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b9053a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f045569 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db6070e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7c593b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c8160e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a2aaa83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27b954bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b2064f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6f7bfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@537eb042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e8f0a3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3680eee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7177c34c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@572d8180 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1135afa1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2073ecc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f512629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@544b8bc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@683f6e4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5795da2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141089,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3113ce9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@390e46e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad00a8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53edf154 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c259aa4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3757ea86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1df932bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21220706 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7182593c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79099e50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db12f1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c1a1798 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fb2e69e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a7c1c26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e911ba1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22fb773c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a15e3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60a25ba3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6880805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f33dd7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1adc73c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68d14c72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e5fa94d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141090,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe651dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@365d21d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@540d59b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb87ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ade4e2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d2b316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b2c86fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40013472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48a6ad62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78dced63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@738afd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed3b9e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3363d0e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c4b841a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e4fc848 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d0048f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10497b65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60525622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@704fcfe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f3776f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147eae24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ef09cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141091,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6851ba75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca5f15d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d4b6363 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63f4bc74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@613206d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@358f16eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f92b42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55283255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aa760d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7520f24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dab97b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4905bccd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3855dcf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe9cff1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efef394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a79c152 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cdd26b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d2afc9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6491cafe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6066f46f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@553bbe7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9f765b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55339d6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141092,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a1ce29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a61485f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e08dae6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@799e6b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc5a0ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b66327d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@331161d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bd5ec9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d31d7fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577ef5fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c7490f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38211ed7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4c8827 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bbd7791 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b6ab30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33d1ca37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce371da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c728e28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aab59bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1796bead -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ec7024 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8e7bddf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78716bcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141093,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab0c1ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39bef798 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c82bc35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fde8179 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23bf2229 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7303f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7409797e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f57ebcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a33066 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503ac3a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc78258 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f8e15fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1312306b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15b38622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4635138e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@194bcf41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd8356b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12d322a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c94fa13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be20307 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@614b1a7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a86c2f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3119f9d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141094,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac83ecc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23e9571a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fa480c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18d3981 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae84922 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@335084e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68a998f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e552 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18390d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ff9ed58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb3b670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c1ef821 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9eecce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6d4928 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a7b68e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@515f527d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a8c3c04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c7e1e74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d334768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76542a09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d2354db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31f26ab5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141095,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76bd63e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@216e0b58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@569c1622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74996986 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a686b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38436415 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e584839 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3132c52d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7c56af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f3cf4f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e964a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24d5c25a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a923141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a193d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aa51379 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4068cf65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bcbe610 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@757ae59c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17260e91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1133f8f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356c67a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17982293 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30edf477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141096,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e9746dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68105ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4a64dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@615b712e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65fe6276 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ffb76d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627a9f7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5892501e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e6974a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af09e02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c47734d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@661fc47b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63563024 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dc8a2c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a026b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72a6460a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20239f6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c5e71e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e7ac28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b13008 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dd5239b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9fda28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141097,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76c5b5bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e5cea1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aa1d3b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3b0a52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65883867 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3d2adb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38657e9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e6a9a5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a05caf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f5b4c9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c59624a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566bb3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed99a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17c70acb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b714b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b64c6e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@652a4e84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a9481b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@115e3088 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c209fba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b49a8d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@653e7144 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@504c71b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141098,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de1554e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ba8552e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0abf4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e1591eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7969f642 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4051dd64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7899eaef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae1b6de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c0c9826 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6278ca3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55019a0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c19134 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ac4fce2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e49febe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7741d081 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3947b76c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14c96f4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7514df63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c28cc98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@850f6a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57bc7bd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@480598a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e06098e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141099,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@416c232e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75baea69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7370a15d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d448cc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d91bb11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fdaf568 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79d6d787 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505d46d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b436fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289dec37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67b87c4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d69e580 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c3b3b07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b8092e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ffeddc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da97059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289f9fe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c71792 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4ede6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48534244 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c67f479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43c30be0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b76104a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141100,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19e52a02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a02d17b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@614541d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1acae9f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a4f7593 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eae0a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d6a7a64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228f42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51109138 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e892729 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a45f325 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f093a04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2971ebb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e1631db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bef29e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4da0f184 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b2aa7f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24fdcba7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff6ece -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c999344 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7232a800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141101,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38d8e512 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45ac65d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@452fdd9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@223a1f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5db58eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b11ba74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40696388 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32dd02bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@436cfa6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d82a6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea67a7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b457f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@402f768d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3450afae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@392846c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f65adf6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44452789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d9cd70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ebc8b5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce49d63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31708019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a79690a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6075fd69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141102,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18026b81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4740d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cb75d4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23667b01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b3428f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f65d689 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69ee892b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79405964 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bd5dbf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a158d94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@472fe02c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c5f1ad4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@593014b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3f5b2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1729e061 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e4c1ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a28f29f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9b0efb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769d781 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7277ad17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f0605cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ed3801 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141103,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71713542 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e21316e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31067b03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@607a6012 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1859927a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc46a5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d18c2c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ed15d87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a4fd665 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4655d3e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d303716 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39997a52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d517ffa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f3272b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49572404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bee8c96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc69abe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41f02852 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3177436c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e32f3e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772895fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37da81f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c3dd841 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141104,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f9c19d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66cd099d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57be998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5756d6a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3800f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42377903 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@185a6ff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@795869bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@756b89ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6d0963 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4783e2d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dac7e0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5423aefb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3061930d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@682c4a9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ee2db6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ff301e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33604ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9c4c6ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e40c9bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@291b15a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab97519 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35e73144 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141105,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d620b76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c25bf6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d651a4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@110465b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c90e55b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d2feb61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68ad25e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab6c49f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@613758af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43e1d30b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e234f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e1c820c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75b54540 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3881a976 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b175739 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2151b4b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e916a9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4beeb627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a919efc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c97507 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43f91cfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d18c528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141106,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be8c209 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1674e33e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1de103e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26ebb4f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b3ec2ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a62bde7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f89575a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54be3274 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2968b9e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6caa4ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@639c00af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4614a33a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a5f9585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79639399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7af97d62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19170c74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb2516e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a28ec5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8b8610 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ade5db9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67ba42ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a1d383c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4de19469 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141107,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a2bf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3000aedd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53bfbc39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123c38fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ad9ca6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736eb4e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1e90c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e168070 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f471b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b4c3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0a0465 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31050474 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16dbda5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e591e20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@358b02a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3486c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64f5a82a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@625b128a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fa89271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a20fd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4edf43ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74aad7ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141108,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fdaf812 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2811e4f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce59ab1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed23b76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56204ed6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71b13eb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b1122c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1798e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a952d0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42299810 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc0d20b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5942a334 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a2c1054 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@444bf819 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49162a11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5017b04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f056540 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3495cb89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c2b39a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42fcb272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4657f008 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c84a2c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49772689 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141109,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ebf72f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17111898 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e7a6a50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e495a78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf4ab8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14bea7b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29ab779f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de56248 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e30eb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3201a04a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40e42e1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1645f36c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d18f181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62a41998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3517f28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab632bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20385893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6212edd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a327680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619a92a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@384a2c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@760d8b21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff27516 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141110,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a48e1d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c7ccef1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6321adc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af485c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@674fa93d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dbee863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bcce5f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3528f38f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f34b256 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45a3ba23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292c2410 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bea1e31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c6b118 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9ec7ba1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a8c228 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab857ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e127184 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c93694 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43f715a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bfa9fe4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d9594e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b29ac97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@88d478 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141111,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f127e32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@233d47b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15461a53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ab2797 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fc4303e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160dc48e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bcc09e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a1dee1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@479a5c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f4bc9e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aa656a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc7cbc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4829af51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29cc7b6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7107038d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127d6f90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@172b09a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5df60f20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a47478 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ddc9893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2275b2c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1726a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141112,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78279c8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71b68b18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e5ac45c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@88ed718 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c09f162 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9a25db0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21f200fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fba25cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3b4db9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@476fd453 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@762bc9a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78310ee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee1d2dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47154c1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fdbeb87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f6d96ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6feac8a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f20cf8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32c558f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ee68f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78783f93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a7f69a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141113,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e5bedd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@407eb9d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@794392e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6860964a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@372ccecd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2291189 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67184fa5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b2647ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aadbac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10425d40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35024d8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c2354f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3befac24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c55e44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0c3aeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@81e5131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed6429a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d72127a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22d6a05d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583beaef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2485b1e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a996c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141114,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eea17e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7901e648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70024239 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b89527 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc39d2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d0607b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff78faa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e773ede -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@193aeeda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24bc098b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77e55869 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30230985 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e216d65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a15d37b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67eee240 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f29b019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a96b6dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8c24fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee03f8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c4bf930 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33649833 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c80f175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4323a53f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141115,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20425612 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3770e729 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c80b3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12d4fd19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2094db97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@106d3903 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59bbdf0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cb2dd15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a7cfb77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1320621a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e48cdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e44b123 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63ee58df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73ced60a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43fc2ed8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@87a522d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1943cb5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c3820f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e905748 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6421918d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63bfb777 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3edd6c11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141116,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb5695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8e02b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74141491 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@217c43e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@795bd6a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635a90f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@373eb095 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@741eb6e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d78000 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64bd4f81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2645e19b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27edc23a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48559147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@461a1e8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f5a76ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ef8129 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d4e71a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@742c438d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68dd86c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12c1a43f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57e86d4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ce7c11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141117,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45a1394a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ade9d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4829b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c54fa9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30f3f734 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6fa6c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1165011f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3e91bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@655b9598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6616647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f35e26d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b83746 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@387685bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a89fab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@259e35f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65dc340b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698b0669 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18af9934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c4a5625 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48b25fcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793ba0fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d39f0d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141118,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35c762c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ca2d8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d526a38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3fba32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb244ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@471c9ee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29713f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b4dfea0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2605481a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d019b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2560132b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@314c51d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d059a72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a91590 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7575ade5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2664f80b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@372330bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3ea65e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77fc3cd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d9268d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58514621 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2871723 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@148be95e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141119,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bd9580c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@796b7318 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ef5d19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8e58d23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@691dbd0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b4cf083 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72727b66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565a7c9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@522b66bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4f5c95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f87b171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1abc8093 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132b33c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17846ec5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35aa597d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b6cff43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19378d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3555cc65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52531a03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee961ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6426e33c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f8b2d39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141120,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35c2a3c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@115a95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53fd76a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75397b0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5545428e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f8a6f41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@677a0a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5925c4e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@268bbc71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af72f86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6b6b8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a9ee644 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118da752 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21521fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44640d3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59d57f31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@259ee46e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eaf0bd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a46f2c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18183447 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a770ebe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf04d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141121,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61fd2070 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@381e8a04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aee6147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74cc5122 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3a116b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3038ea42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e5415b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f365404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33abafc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44b6390d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67fa4759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ac1b9d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@247acae1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f1e132 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6404b4da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@234cae85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@519355cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f35d6be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5611e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8a8b62f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@435ea7d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627f192b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cd8eda3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141122,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@518c39d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67da3457 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df6cfc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2698902e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16ea27d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2024d1d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be7b2ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@91c565c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e8f0c25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48a3e66e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e054d56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7540567c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@439882d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14df9c5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20840c6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57804db4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5009f3bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38dcb888 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383b2481 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0323df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720881ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fcd6fde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e33498a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141123,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ada8648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ceb6f05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489c52d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c373d59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b10051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25f5ca45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f555eb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aad67fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50644972 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4816d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ecd2396 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9e2926 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d43c87e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a6e1d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4160a900 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff96f13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5caa56aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d639c87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@790f1a82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d4a6347 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@335b6d42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12fd580a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141124,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13279bdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9982a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c0c129 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d85f693 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d3718d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73ee4fd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60340105 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@224752fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b411022 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ad0001d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a2c1a6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22e47f72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5157ab0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b7d89d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71583195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c6df23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca515fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5134aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb562cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dcc875e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f26dee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b54ee1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb756ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141125,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4288c6e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@481f4e47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24577d44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1564fdf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11dfae55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2300b75f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c224326 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9ba23d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62da08bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c6e1f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad3a13b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@398a6c16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b27047a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@789d8db9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fb031e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65b006d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5607cacc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e375f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c523335 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29b10acd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4188e49d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189d45a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141126,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450b0614 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64cd9b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e72712 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@102596ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d83095c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e46bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e9a2ebb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62347144 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77685db7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b4c649 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fd4df12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eab21fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9b487a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dbb5fce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74be3724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630d5487 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42f41804 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45367abb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b4722e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4914bfd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bc2bacd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d57eeac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e91d22a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141127,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5297180c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c3dc441 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e7d65e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ecde683 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54f0de2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566d173d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33da0a51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62f21b57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30a1b889 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e4b955f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da09e45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f7a1805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22229e93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@709106e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19aafc04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565f6504 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@603faf5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18e12b96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5665be52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b3f96df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e7b39d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774b994c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c272ed9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141128,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ba9d8ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5089e95f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b238bfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec723a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b6cfc2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14303f0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73ab1221 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@406f5d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6867f4e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d3a76a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ab493f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65b0a670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3227b6ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c864164 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1068017a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@708bb621 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f94e603 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51b705f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@104ad69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c9329d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74ef66b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d1e5575 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3955dc5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141129,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cbd9f3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368eabf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c32a25a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b12048a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb628b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fcf146b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26965dba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4b2bb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67927907 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405955d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5436bbee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40342c45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@349a2f05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e671b81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fef3e68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed867d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bdf850d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d6236e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b584f5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6437126 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21936b0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d8aacce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141130,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54912c9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4946bd6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73377456 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b634a22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16cb7a0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a9a205b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22c77c9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@732400ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ef20fc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d178b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c6bf739 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1915de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61bbcfba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f2117e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c30f70a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b80b0b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14797c62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7380a752 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5823968f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76888034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294d78eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f01f0a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b529226 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141131,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29e93858 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b2197ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b75d6a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3439a04f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54727568 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@338904af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@265c3c85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9edbd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18802c77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6357ffb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24285f4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d411800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14bb7e4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48cd9937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef96546 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfde981 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64fb8393 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40dbaa1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@256d2412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@611870df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59479074 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44516a3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141132,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8508b10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aec8c4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f4d6d54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c06ed10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42469639 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66c574e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a03c9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a7531d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf481fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@715bc0bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f5edd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62f734b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a14aa7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55e5c3ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e07f66d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@758e2d84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a163618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5115c8f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4030e822 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44d61fad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f6b57d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c64e97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141133,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67f0c355 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13af9371 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16ffdb42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45f8dfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65541c49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a0f3f89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6242d041 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a0a8198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f7960c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cf9ffd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@584df2d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1605ae03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e8347 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@610ac388 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56666e10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da8a769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b2a7520 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c8be34e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49f80e21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c36b98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290891f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6a9609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141134,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ea0263 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290f2c2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e587c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab19b8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49cb71a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@197542f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71461d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37edadfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2539bcf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e82844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@578375f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d5ba8b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65ed73f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3210cd02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ef00a4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ebe7b4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2042e784 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@140e5653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a528cf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37ebc6d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@226968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66781bf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141135,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18b31ee6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7187b650 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62fde821 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@447cc01c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44b8b25b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@841b9aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f0f463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27aeb8e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2605ec25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@230c8da6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@215d2648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73e6eb32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a8b45af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d2d96e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@750a886e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31f15286 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1177df2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@403ce95f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61d450ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a8a4b10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752b51e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52fa280f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141136,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11eb50be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19b14b58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ba8dfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35836173 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ea6b9dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5ec6da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f695bdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214f2bb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aec061d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719f758c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346d6fc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca7901d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6671bf7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc8852d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3384ec00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6373b515 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8924c6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@205b51bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ab4d5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a0e9dc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50e549d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a4934c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44667543 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141137,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a3830a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6965d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63659e0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14de2348 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f589c67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64541386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0e54c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eaff682 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@257fe094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61f30f9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3802bf7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9284518 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53897fe9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aff15f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72bb685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c6033fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@548e7baf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60cec1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc9edc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@117bc27f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368f90fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5782d60b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@443686d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141138,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b91cc15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4153b5eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a01144 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70d027b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75a4bcdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2904b369 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292ebd3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5832e20e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8a0cfb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b3ba31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58a820c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17458b4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19162367 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed6046 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19dade8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22fe210a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@adb2329 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@615f8d64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3e0974 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9b9bc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0a2d39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3323a131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d086b88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141139,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c6ce83a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33f0776b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d74720d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@338e1e00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18080d73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f37a36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1629813f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5514e3e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@daf1fa0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3ee6e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6444b868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16bd7cc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631a23af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ada888b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b1ce240 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@163ae0a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248fae93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6fe459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566d03b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1713d63b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eb1197c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd971df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141140,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5597bfa1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cbc1926 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a008134 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af75c22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5275966a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e3cc02b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@702be5f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@703db192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e05bf39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b00b4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299b2248 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b7b7c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e82a10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118499fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61fd60e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68fd92e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e138d43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bedb804 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d03cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@573cbb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27a837c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@416386a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141141,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28096631 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@333c4683 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@554c6113 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8c7f3dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b9c341a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a457fb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c37c516 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd98656 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31ef49d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20463885 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12eb46ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@657799d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e1de57f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f0e6229 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779a8f10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efa3931 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f8d078c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23de2f7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9fd61a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fd35489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78e58d98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a12e825 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141142,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54c3d47b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3151b576 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@650c6789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5862f8cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12489fb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8e628c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b878fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675349b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31b0dfc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c77417a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e393af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a005c23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4df9d26e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b53263a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b6b014 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@782f0565 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b46d2ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79a65908 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca57038 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c2150e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f287e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a1a5f48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141143,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d59c283 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b3911bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13bda87d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc5555d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@686bd37c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bfc9b3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35fc2c7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1432e821 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22f429e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d7e73f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b521fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a45a90b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@597d70c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27e271e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57281e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2052f839 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@129b7791 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2642416c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310cb2ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a80784 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fc6cf3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bbbbe8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141144,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ff6ea83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a45cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10e2c9e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368b6a9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb13c3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6538c79e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6393bf02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e21d650 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29566b42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@af15dc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74648832 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74f743b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e613a1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e7e6d47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e0b4da2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@702f3358 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17794e80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a701d7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ef237cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@674e7cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a1e339e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@308f5a55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141145,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f3c47df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b834135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10ca1b18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5addb6a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f2009d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b96dbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ebb2ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41344256 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26255405 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b8c01eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70598edb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@139c5474 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f95bf7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24ad663e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183e0b26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6970c3a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bacefbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14fffe63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c70a0e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f21905 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f9fd5da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44782aac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d76fd6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141146,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ecff1f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fafd2f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a07a8e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60109562 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2191b8e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3038e647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672f3025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f01227 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f0ae408 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58773570 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46478843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4525db86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d774d8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4a33e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26b5adbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4da7ab04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4abb8403 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@476343aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cb0c6ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c3fbbb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a32bbad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7711c352 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141147,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d5efeba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c7d08b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ad2b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3adf6b32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70bef9fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64927fed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ecf9f22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f08fb32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8fe8a3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f64f1da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4de0579b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9e47e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79aa298c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2198ed42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8b2a65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241bf9de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd54298 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a415cbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d2e9b92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26804ce0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@670b2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60231b2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141148,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49f8feb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d3b4c21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@553d34f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d3fb71c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@233cd167 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1451cb65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f7df94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a00ac1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dd24b5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2742a567 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31dfc564 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d1379c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66f6356e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@542dcd4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b16c64e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b3b7f67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb16c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@364ca76a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc5335b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b85a703 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c52406c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f76c37b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9b7e25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141149,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b10632a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45cf0b04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c5b75dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a175e25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5256777e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ffcd11b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@86ea4a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a825fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49245a16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40593e20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3defe90b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54819cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9e98d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@363f0938 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74da1945 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c662733 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67f55a5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294de955 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@df9dd08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@103b3c3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dd183ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b10ce42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c251b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141150,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11516472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@404e4b57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d31efc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17344228 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ddcd648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f7a3057 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d679785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5877ad03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33f24ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d643872 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6e57e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6914ecdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4831a3f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34eaa524 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ac5052a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f97ab9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@411227bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20b0693e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42bed457 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ddd520b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d22a3c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e78c8c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a50a735 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141151,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d7af97e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@650e63a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d088100 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@122efc9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68e556a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e603b76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67f9b53b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d6477b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40440996 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f802163 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15fa0bc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a76b9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@454fde05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7a74f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e9bfc7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@207e9ba9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b7ad655 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@330fdc08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5296a612 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1cc3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b647acd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c60035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141152,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@610790c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698f00cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@155ad21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a4c296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a789d73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be0618b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5770f773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d12f79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68eca1d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1078797a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40e3df69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f069aeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e970d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51896a59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fcfff96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3425fd46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d034cdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67422a86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17815ab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd54acc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b578a2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d379bd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141153,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fe3e2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26db97d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@718a77d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5042d36e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21bca822 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ba3c6c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793526b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2a435c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b130123 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ec30506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e938cf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce02da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b71199 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46fad3bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347c1ba0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47571727 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f34ed8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fc3aae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bb611a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2427cbd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c470255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ec10fb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65910ce5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141154,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f726a8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b839a9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5db9aac8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@302273be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce85135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b78674e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b076103 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289b4c5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3664c181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6efe9214 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e41f9b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ccab79b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cb2134a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c648fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@771bb1e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e14a6f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2290cc3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3605c10e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c3be28d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37b3ccf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13661705 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf3b9d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141155,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@481f1daf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e29419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@429f691f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54345593 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64f6d8df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1141da04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@282f9ea2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ec1def9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca3d5de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132dafb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4723e9fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17c5c1f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55afbf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf4a4e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23c0d667 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab37998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e4ac68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1669ea1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bf6559 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3241a50f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2bd8c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cbe839 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141156,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e96c014 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71407d14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ac4e3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29fffa33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51910978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23cb6683 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31840973 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@677c5312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eff4c4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4678ca6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57269c43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5247a176 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@161e1a5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b59949 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7592db58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab1988d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c1bad6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46de1c74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25f547ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3396c62f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12c765e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@244ce677 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b091f35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141157,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e2b03c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eaa3ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4456926e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27c5dd66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5106c770 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3235999 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73167e71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a31a398 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1debc6e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41dbb5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44b8a858 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33e2ee34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@707dba2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19532827 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1828a93d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9f7742 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ae1e3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21bc083e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c803c75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b6261c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aecc76e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b50dff5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141158,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ce3d7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8a3a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72d1550e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f5393d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7851608f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57242f89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f256946 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cab9257 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f8dc255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6923ee4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc8d571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d41dbb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c2f2522 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a08df53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71c0fa3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220f1bc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77dc2c96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb28779 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dd5913b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475e585a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f074bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141159,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e86b8b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f6bb811 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a42af2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c54371 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631d2ffa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21f4a8b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e705877 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cfc4e03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4694da99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42210b9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e0e5048 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@702c91dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108ccd52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5abb710b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769bf604 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622d2b7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143d9c30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@150f9495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8d7be6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f0f1f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f543724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2616a967 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141160,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7113a408 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41dc849d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fdf7695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c91af98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ce6fe6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7593e10e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ac5af7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1070dc35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33623082 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59132c71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2bbe46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c2c5842 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e688f65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ed85e7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2946146e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4746a74c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d3f1b66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@138830a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18440227 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a4aaaa4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f7b78a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a9e8c96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4273fcd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141161,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd361b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1989b8da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@177d7d41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a35d64a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c8db015 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78968236 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd7f10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@601e7e43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d2674d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d19c673 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3213f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118cab44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23565e37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f3e70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b3e0d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f1c5335 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eaa0f07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f021c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc6a58b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@614ed6dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f323fc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3022abdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6991aaae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141162,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e7e717f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20ecbd6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5359ee1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61958033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c6df0f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29af1ff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c45eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9dd285 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e746989 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ff59f0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de643aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eccb769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7399ad18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40275198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd3a125 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34df3049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a90baa7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c181fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2249cc48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f59fd30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@609327fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca18471 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141163,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70663021 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50f6f360 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29a38257 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39b1191e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7817c131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f9d50a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a99c83e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b515d9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d891988 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28e45ac9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd8bfd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d74965f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48249120 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@421ca8ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b33b148 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@328c001f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ecd2aff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b12a89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f5e6d06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d001c0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aad4b87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab8d539 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141164,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6652b935 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4f09b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b1e8824 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47f39363 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@341b11b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577ad442 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@186a2921 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43d9c9c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fa39edf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efa87ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67909072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a0af2ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@596cdc55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23d1ef3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40a1e832 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d83f511 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b20080 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e368fd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21190a5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43505810 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11b4e6fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ba6bb08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141165,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22040bd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63555759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5560eeff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b6071a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29b8bd9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@609ec86f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@325bd595 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a1e4f0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a6459f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606d2694 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c76a071 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6478e0c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20b6cb6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29e6006e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10aa480b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33f7fc01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@250c9c8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c701d31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e58e0a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72431c9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@336a06b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f4f0c79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@362b80f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141166,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c66fd67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f5a2fdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4881c08b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2247d54f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49a199a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@276b738b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61287a75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd936b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f0e41d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fb3fa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66220e9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52fad84b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d950e05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@708e3233 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62eac6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2921461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4732c710 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ac6a6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3592fb01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27b1d0ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ea996f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53307acf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141167,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d8c95b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5846400e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a458b88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a73600d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dd6f609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77fa938d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b55f31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10f73002 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3abbfaab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19fe708a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12b9f853 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be7bd26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eec6653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34fc2724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69da2a88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f52b418 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5231995f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a1413a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f17b52c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@451f24a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bbab63f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@169f8ce3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141168,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1423f5c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a5e9db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6660a9be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf2d905 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f4f8ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1822fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63d6561f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79f68191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59652dc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18abd3ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499c0849 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42be2e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7450d48a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a24556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b55dd0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b21071 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32347869 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@129c138c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27943951 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f5944a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66135ded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@146122d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141169,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12b9cfbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49354e65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35358e9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285f4d4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c7b3d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17628d4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e3a986d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c5d99d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15cdc087 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16c6379f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce52cda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cc6af8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f48168 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@202934c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45cf5fe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cbc2a04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe0153a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46868b04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7909c446 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14083d69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3beeb317 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4942d1c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141170,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ede175f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b688e98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65d0f6e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ac3df6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34295387 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5a17d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@395fd5e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d435d0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6308904f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68e7cf4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f50a72c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@287c0cc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fd17b60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@591a5070 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73d9e5b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b8bbd47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f85cb2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3417857d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@85f5262 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e58a7ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f95c5ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77218b0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f0bc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141171,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed1912f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f3a9bae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43806ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79048caa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bfe224c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b22576 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347b40b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69c7fb41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43814877 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22794ce4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce9b693 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2241bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a9707f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5f0882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a836c54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a4e519 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9641ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69ce4772 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@480e67d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d97cfe6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4594ce7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e7af468 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141172,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6571208b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a2b3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65fefaba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c2e9bd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@379baf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c3735e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@457a6836 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d92c313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c4c1ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d510708 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bfce14c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70bd277b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37b14b28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127d7391 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed70613 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c8764d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a1d8f97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@130588be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6514f353 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71fa8ba2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779dd33d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e2e508 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a2e092c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141173,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f2c3450 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b11eb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@692477c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@133974a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d794c2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638395b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2953343f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71179180 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66188b94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1949acf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290b8128 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64dd8e18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46bbed3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1241f725 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67362684 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f13d934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76580be7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b3d32d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d693b33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c429fd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a73b28f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@785a8495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c9e980e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141174,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54fa2249 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337a1593 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fa85145 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@670ee456 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e37a94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1563dd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c2976c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c951547 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355bf19b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d3d26b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12db2b67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76003b56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cbddb24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e60252 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cad81da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b80d992 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4996513d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed52ecf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e56d39e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a7dd8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c3677f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb781fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141175,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@279561b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a039b6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7978f3cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a33256d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a77557 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60195e1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ec12f48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4a71b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@707aa883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a39e927 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dd53647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@138c01bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36979dfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23722ada -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b147fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c33756 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eb12c5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c27fdab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a4c57d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb02398 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b942197 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28ad45eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fc5f2ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141176,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6244b42d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1826b96b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a663599 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a56ecf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df4d242 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50b4d176 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b15fc04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36a9febb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d145cee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5d1f5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fbc25e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76994e82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39258f93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4af1f82b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@240c17f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18ddb0a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d11ed3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48647c13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c131130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793ec08f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc49aaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@102bf520 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141177,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a5fc84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b907e27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54de8c18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b13f156 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30796f6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38904e02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@188eab44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c937042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@676a6eb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5db0dc02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3009887b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371fff58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33410fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58bf8c56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24898d6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca4439c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb41d16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b4de58a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347c754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4db1dc03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2282240e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2667f461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b27fa9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141178,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58b57c3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b248171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7279b488 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@562e5353 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2589993f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ae9abf6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550b9fd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719bfaee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1832c583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f236b68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@365fae7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183d6ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16adae8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3271ef1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79968b60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a2e2977 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@710c0f46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cb81774 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1385808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a26acc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc1eaf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@674e434f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2475afb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141179,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b7a1f15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f08dec4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a49b051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1348a4b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f1937a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6146965e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36da691f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e12e59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@442f0b7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118e81c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74f9907e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d03eb5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21cb7769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ae6a18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6c7893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e37255b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bd75cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337421ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c436ff3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b199add -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5b958 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160c937b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141180,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21874abe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ff321 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f69a5cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0c7c10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c50fb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f4a1ab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d6f875e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@633d8c1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d6b9f6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@621bb3d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ef9db8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3dacae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d7cfa8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22464fd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dfb147f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c0283b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d0b624 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56f859f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13e5e7bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd2e962 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@787019aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cd3f096 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50373aac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141181,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c025031 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c7bb52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7763bc1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e680a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70054222 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49a17a5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2348bc2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7daeaa7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27e0e3bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@655136ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e47fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bebc098 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b362741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47be207b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@366e96e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63d46fbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@794250a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f491a02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d520cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dbb2345 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2617f543 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21096289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca0e6d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141182,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28954b03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31b10ebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b463f2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60eb9cb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62241ef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5930f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310cebdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dbec77b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fafcbea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5de39472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b67fe26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7019752e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4158105c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@594c01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347b04e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f70b746 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eab8c04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44d8e094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2868b0f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22a8f835 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b19bc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736e190a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141183,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eec90e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be0ba5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f31131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d6ee8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e472550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b428c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1feea135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cff8eca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b5553a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3190e364 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1d0b6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc4571d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4acdcd96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cf09c68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f058ec7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66505485 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@425594a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3abe535a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@592e817c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dbe02a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a04107 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bccfcaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c4abce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141184,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ba214b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2adbcabe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@560e5ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45071137 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e6bba97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a911bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e6c9b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31fa37e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70531e88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@224ea4b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3861f620 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@469ba601 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e544704 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79e82565 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e0f0c8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55bf00e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141185,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59df0134 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6124c467 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5779dfb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a6c6dbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1b0566 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c5bf59e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54625f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cdea978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f6dafb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c594ec6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68849103 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f87002 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@731a4fa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65a30079 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29638947 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd98409 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a9fb18f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3742d06d -17-08-2023 17:39:01 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2de4e73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141186,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25806bed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1224b51b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33c4de8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4319b357 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e077da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52db8a10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477e3e4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49f3aefc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f4b4ebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214eca4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17f4146c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4324cc20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@366f5c4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61a4fc66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56c4d34a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3364507e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f25ff6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39247b25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490aa1c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@587cef37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f28df4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1215329c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141187,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e0dc5f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31378656 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db620a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160c6651 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ce2b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc616a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6769c21a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12a21e48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@153361fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e0b0ef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@775b7e3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75896ec2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bbbd507 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d411838 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c72b29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f06b0ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab053a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fc50211 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b3e8a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@deab920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450fee51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@524db96b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141188,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a73a571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42883ca6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78cf3fc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47f64753 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40293af1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43c7435e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@427745d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@374e4e1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3904b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@479d7d9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fbcb13c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac70e46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cad8123 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20f04488 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a9f34fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38e8a0d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58ac9a77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5a51ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79619ab2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f24b6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383106a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10ccf56c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@658d49cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141189,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7332c729 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4386f3c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3496094d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bdaed33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eb47eee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad8ae2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e800658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@425fc5ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f618065 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a7f870 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e3b86fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42f64d46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a79fd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eac9de9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c02d490 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cee322b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a5554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc3d347 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@514f2efb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c0d3193 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f8e19b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab2acee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72a269b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141190,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77264fdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@382a830d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77308d3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16bebda8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a36033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ffda3a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd7b039 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe707f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@678d48c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@353f0f3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@441827dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5357401e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e91d1f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@131a53c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b9b0ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f8745c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc93dbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8cefc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35ebbecc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70874cef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1645b385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fbe945d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141191,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77cb54b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de18e9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@603db38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6df4abd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b75c4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71939f99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69d14088 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b38408 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1cc171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@361ef2dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55282197 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68866cfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@552c2fd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1465bfbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d5103f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bef6d47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f89bb73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a86968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e54df6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e7110b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c816a2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6977bffd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141192,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31808236 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a2e034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108536e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4459fb5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@717d40d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aead88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3825995d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33ad161e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284da360 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cfb890c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48801cbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1346fe57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565509fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566659d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ecbc244 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a5b338c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44bfc7d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20f3b657 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606a89ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e113d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f5d7d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4098fb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30185b29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141193,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@432c63ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54706f05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c4c3e02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e7fffb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eecd283 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19b3841c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64f20506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75e1fc5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d3150ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58a7ac75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c894852 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56705291 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@113e4fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b234af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29f3bb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56af05ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48852f86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de76d75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@465e9bd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68724ce4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@96ab773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f60ffee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d523d88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141194,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@473d43aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb0b511 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13a7a606 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fb57787 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123843c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cbe74a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228774a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@361a263c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@666c0534 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19887e36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@331bf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67fff60b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61bc0a65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b1fa1aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@236be99d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3c1c55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad2c675 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c063274 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dd1b68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b586192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7795f5e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a4decf6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5223a68c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141195,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a6f19ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@120ddfe1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47f2bb0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@82a62be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a3a9051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4634b220 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14cc6b07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f28c4b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@217e7d1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b86ea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355ba1a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cdffeed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e9bf2e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd744e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769ae18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4402108e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75bfce31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@201a7c39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a491f4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e9be4eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355e7b4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141196,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d62269 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64c7f680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51afa4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43d87da0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346adad8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f425923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d581e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ded4170 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f33e00f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22c5218b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f10a3fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8a18fa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4852252b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43908cd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1945e021 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d9df06e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44bff58b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30e1fdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@357cb373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47902b32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c388de6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141197,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fbbb57d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f8c78d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@190311aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ece8d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@732ea1cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd81d93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed7db83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@83b08e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9df8e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1264827 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@715d9043 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2171bac8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40ca1cec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@731e03a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6edc4a0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42570ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@526a4339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@516a49c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5deac5f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2790ccb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@276320f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24c461bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f8ab5d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141198,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e5e29f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3004cc4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@374a8272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d14cd3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32fa6cc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248ab860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4730f791 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c26a77d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d5b6271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c639a02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7caf6e60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@652b33c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73f58df9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5366ba0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54bd026d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68b4b636 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f1749c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512106af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d33c88c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@207bd61a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c92031d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45df6a5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36dbbdc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141199,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@486e7606 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44681efd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5b11b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11859e8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a6014fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7b68bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0f0a76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31c3acba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1baea1b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e6504f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1acdeff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3647266b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@580a9a1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0e774e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b498934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b29e48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68642aa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd94e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@221abd4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52aa3799 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd23e5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72218365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6b0b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141200,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b34962 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c19af3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72dd3f72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@424e32a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eaa9bdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f0ba766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4363d5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37de8c0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3da0ad6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f521d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4abb2374 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@94660d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31356751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e619f0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d52b730 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77284ee2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d3db0b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ff6f3d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6745382b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52af257f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5217efeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a77ee18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143d011d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141201,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f924a38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52178296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f5a1e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bbc6be8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66523b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8bf1643 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@125f5c07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a94968f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d6849a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d954343 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55c28f92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b7c4b4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@445626d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ceea44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123151b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3e76ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e2b9789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a13d8c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b7b8e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a743b1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28717407 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c507c03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13efb7d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141202,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8d8ca79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b2a5629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7445872b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1afabf0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54bf3528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5daa88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75f91cd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a34cfbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@170fbba0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57464a23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ae4ebb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76b1b848 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c89df85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797b8965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@348f14d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f0799f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da0026a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3835901a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74507219 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76bfd32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac51ce7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e64eae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79d3918a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141203,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@210971ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1319730d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a038049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fb91942 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35558d53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0c01ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2bc5cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4fb6e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4569972e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4867e515 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d77546b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cae199a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@757721c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24f2d30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@657cf51e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@691242e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a73717 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5efab83e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c2c62fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27523ecf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b51aabd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30e1dea1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fe541f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141204,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f7b280 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f45f341 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31ef20de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a025ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1047e659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3038c0eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7899efe2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d83a2d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d9f414 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34905b3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34144b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a4eb32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e50c806 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48af254b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c29431 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@156b1a40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d03f013 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb70585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@981713e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4475d354 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368430a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52fb80d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50f11339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141205,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af49500 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2de2abc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23111bea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b21ac6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e36a9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d675973 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7602c77e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@367728 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e7f7207 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1934007a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@796d0d15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a51ba27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbb0620 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7635f87d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6721560b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e6584a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@377bd070 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e5430f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8b23d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@119e8e0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@788f3419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3713485a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141206,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71ec2ddf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14670336 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1549c25f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38b51934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a74928 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47591b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff5f052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f192e21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d65c0e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75357aaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@532633b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc2d9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@704d87da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e736b7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ceee30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a1ca44e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46738b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736e30ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9eef98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31f0291 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59eab6f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b033e07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@102af8d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141207,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b5a6bce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@670b5522 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25babcc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62913cdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aad2f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67b2c368 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@179e2ff0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c029e0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe6b92f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3e9a38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58b9d3a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3023e473 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a760c99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53b0b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e157cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ce291f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4326f87b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eed2eb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ae5d94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a60120f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e71c618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@495eefda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79ac4998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141208,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73d6dcc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36785ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d799a25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2682f49b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63f75c1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c82ff28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ddbee2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cccf123 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45025101 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e329d36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ceaa049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@754031e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5089c0ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ac82ca6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6309255e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@446296d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a70b252 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d618a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bdff417 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b5b39b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74e6eca3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1594c9b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11efe858 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141209,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630b9524 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@204bfe12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13a162eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de2e5ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ffedc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d3b30e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2595bb97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb2ef57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@729e9a9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16c61014 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e074858 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4697d498 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405bfcbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66bb1211 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42dad316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a11fcc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b05931 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b25fc0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff3de1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a31ccd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6f7370 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299e394d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388c1955 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141210,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cef25ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6afd33f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f13bb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a6fca09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f5a42ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cbb9057 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@272c0cd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3387b5be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e07548b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e7e4a34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79738ba7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77d7eb72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33305a0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fd389b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a181b59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52dcff13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630d2765 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@709482 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae88506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff9cd2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6545f9ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6115b514 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcbcc85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141211,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5efae6e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14b85caa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c9f63c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7947a644 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7083a25c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2608edea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e78701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@755662ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@182abf4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@792c7619 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@747e39b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f9ec58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe4cdab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e2e0f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60d4e657 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d325d39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e242a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f5cfad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d4617a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8021ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@282183d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30b1b9d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62bcc104 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141212,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dcc03fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@395714ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4151d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25484e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d0b7847 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c78c901 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50287ff2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ae52a81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c0abcda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6600569b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c47246 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a6ff5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc37b3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b9354d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d1349a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c5e114c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f0997b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69146f73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e4d459c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12e5fd68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e62950e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141213,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64470846 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60a2c0f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d116f36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67e14b7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bb87afd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4deba005 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b550e51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@768204f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e15f748 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6e8507 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dfa271d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b392be9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab72a71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9bcc9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c25d15c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e41cb52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2df69249 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73575c4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@212adc49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c19865 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48bc3cce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@140bcbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141214,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76fff04e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b205c44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2534fa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6728e37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f066337 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26edccef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e5ba9cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c4a9b05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10e95a60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58ab7626 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5db92d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43b78ff0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1957f7bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a4ea8f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c8cee43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@333b202d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1577dbe1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a11d8a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c4cf56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b14a70b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1097db5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6437dd38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@137e1097 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141215,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dccd10c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@232f9eed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@792b49ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab214b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719de25c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e9ac274 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ddf902b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7423a8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1953b73d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61a24f01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108dc5d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60d232e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3868d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71ca0b97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f97d30d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606c8582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e25b0ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f928a0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eac2548 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ae8ed8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@399982eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7210ad3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d194051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141216,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503066c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@254b38f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52c50838 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e9d1f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@363c95cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a4ca752 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30b2e2eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bff9b1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@493fe740 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6666af02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4164bc9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a4f0dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@101716ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@435aa511 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@273e79a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@261d4783 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e978788 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18f3d966 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd458a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393fee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@322053a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@349e946d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae19b4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141217,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13ee19b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23e3d50b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631e10e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3573dc20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56b368bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc5a4ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@784f1052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53f651a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ecd25ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5869101c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58a446c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393e4f19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e2be79e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e52e7b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@243a20e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb7f5f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e31f886 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3797d7a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42dfcc9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1217e105 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6393ce09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a306f44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5637c584 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141218,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b05098e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69b7361 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f3866b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74b953f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d1a521 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@98cc4a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252d7687 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e94fd70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2735a72e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4873c6a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a40862 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f021abb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5584caa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3947859 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d59751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a4d8692 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@196f80a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39197567 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30ff120a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36807b91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530f8f7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21cf6a31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69685340 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141219,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c01f10f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73dad1aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299eb302 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2471722 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@519d17ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74efcbf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@283560c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12f18ceb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b6acb2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d015dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22e0020e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c68446d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4126cb75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f94878 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@422bd61b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c1ea91f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b14a08a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61848b18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d83023d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@349c0ac3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae2ec2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141220,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ffb7025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a0c1b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e12dc9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63dce6c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27e2c8cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dac6b4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e2be51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18cfa8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3019fec4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e311d89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e313628 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@188fcbce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671fe9c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1247dfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76538b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1064ff3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@82d6d37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ac40a65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40a18b36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770b191a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141221,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@142c8619 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb131db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5205f100 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de9f193 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d85ac79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2d7bfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67dd144f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@558a044e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@754c9ce2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fd313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b3fc0b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e121de4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f282b48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6585b4f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@104a8af4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ea95824 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@508550d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18ef742b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3650e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189a4b32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62639b33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141222,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cef578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f9144b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@251851a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bdcec31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4fb707 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@481e9a9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f6c477b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@136d8bc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@434af126 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f3c6196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a6a695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dea1091 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59bd0bcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4511e1bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42046582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff79bcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ea530ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24f318fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45ba4192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26f0a32d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141223,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6197df06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1f270f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3e5bef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6974704e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c4ed89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12c79301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9e5395 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6022b088 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719ba1a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fea7b09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0999a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e3df14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e59e649 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52e7ecb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5198c337 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3defbd6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c156f35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@417ac423 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ec2640 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1628fd37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11c35400 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68fc1c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141224,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e62970 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6076ab3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b9e9064 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13e7f529 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134af38c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b08580 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b23082 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d0d820 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31428385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@739c8ca8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be28479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48805afd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7bb6c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a958e08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55fb0c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f66f39f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f1b931e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7928ad3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74bd4e05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c6fb106 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f9f5af6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13e26b4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141225,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@db1c254 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c77b565 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79ea02f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@795286b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7af248ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b3f0608 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42eb4c22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f6135db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@473eafa1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54a2ee08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3022e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a591952 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58a8f01f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb997ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d3810b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74e2910a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b55af81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8651b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769744f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@161de1eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50f27d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141226,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e77ad71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@120bd299 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b607b8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b961e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bc64da6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@715d15d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc1561e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c14b44a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75467313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cf0dbdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@396407f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10fb427b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5663659c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c9772ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1555e708 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff78743 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127556a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25c04454 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f979a43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c39db6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44417238 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141227,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63047c00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4062a7a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b2ee73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d2d92af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d091d76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2883e2d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d2740cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c4e0b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2999477f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ebbfdbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@696def84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51696e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66578b85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2740c4f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21570e56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10471535 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a16bf1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd0bdc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72ec3b52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a2a7687 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@253be4be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141228,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b6b6364 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58cef7f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d93c20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b3d3cb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5d1af1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14830b1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c230d91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eb97010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6071a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11b75af6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56c82d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@504b99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e23f766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35686ebe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@547320e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56fdb571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57e4e00f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@210f79d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c36011f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141229,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57db67d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27cf01b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d542572 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b6ed3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5fd312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72c2cdc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e611956 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38a8deb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b3c5dfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7479d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29f96ac7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57da3457 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e8048e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7216302b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10dbc3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16a5353c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3e9864 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@96a6e60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47609892 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78534997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46100341 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141230,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cd36c8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f34c38d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc9260d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8530fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb59c07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56410b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7169b758 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b97b4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733c9e9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5678d9e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a976707 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c3a3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c38497 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@564d029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@549a20dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37b4b5a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e5a68fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d9c1286 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141231,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18854520 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a0ef22c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67fe6619 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a0b5042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2815ba66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f7a2da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@412dd8d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e409f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4275524 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6caed608 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ea600e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e06a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cd08d23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46540e27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56aeecf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37fe5d12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f9f775b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9dc79b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b4d505 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d670bcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79569c94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141232,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132d23b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dbfd02b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4853813b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ac1699 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f66a509 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cbb1c46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48b50cb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d48d3e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@674c75af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23510c2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43e14c61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285d503c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb8b995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@723e042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ee83683 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf643d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@656b27c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd2ead7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720197c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c57b281 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70be89a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad04136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141233,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64916a3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a30f92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1154572b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9157dc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ba1e6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b6842b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5836bdfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aef0073 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c903035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6544beea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@87036f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a9e4314 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4742cbea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7318e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e7ff7af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76018397 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27cd0b26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c07e067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5db4a8c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27d385dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68882e3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141234,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9fef94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39592b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38d30272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118312ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1564bc47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3acbac49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2be9f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13349e6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23828ece -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ccd713 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16dd213 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@369dc393 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dedf70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2388e417 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9a573f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14439464 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6391fcc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d04fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19608881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1108df7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d935e7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141235,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1206ffe7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f3f49a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@581b9c0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9c71f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac05826 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc750a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb86296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f7b5cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2feeca7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7af9c579 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527da462 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582e3a3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14abe67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35ac613a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@acd8d59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@239ba1ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77768a7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21858018 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47447634 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ae23ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac873c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c0a9c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141236,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d184db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ba85438 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62477afe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10bb49d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f53f046 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16437c00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f0a74bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b57172 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15746042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@185abd90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e995b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a795abd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cb959c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5142c1fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19ce926c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79828f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fa3bef3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6829f7ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5195577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141237,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@680cf2d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aea9d05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5b9723 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3226c712 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d3331fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@333f81eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127cea20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e6b12a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32fed6d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d0129bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@859e1d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b156e15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f1965e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b21f56e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb1e896 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c5450d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c610c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c580034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ffeb071 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d2bd95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141238,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@95add40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252f35ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f186413 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bf25270 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@641d3893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38417922 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1455aac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d9d0bed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c9756d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2823f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ac80306 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fdfa5d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1446c75a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3325c97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7998afc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b7b6c0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49251a29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e2317e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64384329 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54137091 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3506cc68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141239,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ca66277 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa157c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@157923fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@161fd8b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5965a5d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9aaee6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55905a61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d22fa04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@491dcb14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@171abdea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@400eae3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a568ee5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d34bdd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f03bc71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a49bcd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64363cdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c84d7b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e760261 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69aeba6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3caba340 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b8fa16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@387387aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141240,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24123700 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45422161 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147b1b95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30919067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e0b42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51137bbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1df865bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2156dccf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@547707bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e73602a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40783bc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57df27a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b798002 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e0ed577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f296f7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1165bd0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@666b46b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e069262 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a20726e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@661111c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5af3bd5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141241,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@407ca1e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19961880 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7767b547 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30b9f7a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@156f2bee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9463db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76133483 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@968c10d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6534bb95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aa19653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@157912a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad6c9a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a7e397f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c9b276d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@312a8404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d5dd853 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3603a746 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3965ea4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6679e076 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@555533aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ec787d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f9595d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141242,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6646b607 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af20106 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5849e486 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b91c951 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68b4b6aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6244608b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75bec591 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d05136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d9cae60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c1dc760 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f0c113f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d17b009 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6590e065 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47141d70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@683e7944 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58990bb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a684b0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d703eac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@469f981f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@564bcfe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141243,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e83de0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72da1029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@483f210d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4003eb08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c00685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52390bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@633f6fef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cbb832b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3f5e17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56435a17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e1b0727 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e597714 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18204b2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54ff39df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@424b2e58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27cf00d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dac563e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1030b195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b926425 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54842c15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b469d1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6f29f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141244,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c4d55d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7355b514 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c446016 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b03b886 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b4dfe86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@556e07ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ac59985 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb4c1d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68e9b70d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9d8c89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b022d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7112c5e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54e65514 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@753e62c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f1e5b31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@584484d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e6c343a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38c25178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c85282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d622f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56440a10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ca7d831 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141245,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bccd965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73b5836a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d71386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4773255d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad9cb27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a8bbc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@85ec5d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5659504e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@727d2951 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fbd248 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c75cfaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60322b4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b1eb1b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27ab4500 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75f1eeed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@468cb4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a752102 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770bf609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@df7f883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f8778c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141246,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cd93d02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@444a352e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76cbeec4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fec1ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@276acddc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@329c6f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b881a03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f0f5b7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e7a6f6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f4d2545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14ce76ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18f13e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cc305a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a826671 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d1e0e92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74775c8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20cda1a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5905ea2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24120502 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6acf8e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27c2b216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5160785a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141247,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@656e4d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73475074 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41270e08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13570cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c55e90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d486a8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34667a63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f7cb1cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31635d6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f714dc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38c39d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42cbb125 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69681c2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc263bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78cc97df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6711437a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e31a2c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a1cf07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dcfd283 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70a848d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@334ccbad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9af334 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa275ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141248,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7e5019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38baffb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d2a8c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f13aa59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f2df3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5ab7a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2908c668 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b4bf072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d56e12e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70bbe88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68df0387 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce3b68e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b67bad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c13780d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc08933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d9613e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@785860ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f5cbcf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778d2130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a692d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@210d7993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b7ca23f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141249,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75e68ac1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2635e131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c3ed585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62ddef67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@232cc17f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a198042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3011fa79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dba8dcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7858943a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e2500e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2158e36c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4865e659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74a0422b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42346644 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2491aeed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4321c308 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3566605d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@548136f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d4e1c48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@303aa96a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36d843b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb3cd3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6612205c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141250,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@386b40d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7737f246 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2981ba3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ffcb6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ef3f02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b8250fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4905072b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@422a01cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@780721a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619bb3d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6dc90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12709925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53bb80ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f028465 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5916e749 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb9f4ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@408ba036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5125ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5513b162 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550db933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489a39b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ad60b1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32062d85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141251,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7aded7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@133411b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f07624 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a19129a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@649e368 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3543e3ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd8e277 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d47a0d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345c5eb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ea88d1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36cb0fe2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ca48ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20b50967 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aaf8036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1475249e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30dc9bcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@382a5282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e82cb81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36e491b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b8b85a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d6c4e6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@159a5482 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@644de1bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141252,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a5ef00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29e5d89f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1ca765 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65d7c324 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c53fd71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b75fb44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@305954c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b4c7f0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4857af61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c6cd2f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@266962ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e55cd33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b8c533f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61cdd1c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cdeb916 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce2c1eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35ce6ee3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a02aa03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@436db25c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@591ace43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f651af5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3018f9a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ec952a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141253,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11aeefe9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@262bed9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5109854f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@501b9b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b6df5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a88b187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7367c64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f728102 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6bfb94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b06dd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31d1e459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab019f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f82e069 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29daee9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@171724d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d42d05e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fbee37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b2b6625 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5534c419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141254,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e5f3ec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248fed6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@435bd91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7752b502 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@707c341f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bc36bce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2338212 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60abddf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e6c135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@414b59dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff670f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618c7abd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a4f0d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dbad861 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512776fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@537ca7f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52dfef3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@394d4cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@533765a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65420b18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4849b698 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb160a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73435452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141255,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62c9be9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35400c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1370d468 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a78e390 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1869cce5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b989d3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e5eddb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1387df8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@455f27fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad11320 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62bac2ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f6c02d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73b0bc01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41cae944 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d15cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a048202 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ca26df7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a3431ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@457ce589 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b167bb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46474f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fc48ed5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141256,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db67009 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@357577c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1291765e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2918e0ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27a0ebed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f553bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@204ff15a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ebee2b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ee1970d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d1a51de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2200ba06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b1247b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79a9e220 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5692c86a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e59834 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78436e7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4d3bb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e5c6b8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a57649 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30f6cad4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17c1dfec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141257,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a2cc681 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3f0b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4653a55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d42bb8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c66e6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c9d508e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310766da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3991059b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b96a1c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df9991e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774211cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ce6f63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78e85ec9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@496809a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e43a53a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4362e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21677951 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28cb59cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b924399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b46465f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1710997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141258,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73062d57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a012625 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53dcf384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6875d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18469756 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b85ba75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f712452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@462e44d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a7e976c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c6ece44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a6b3c2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e57af3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@508c846f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a4a6cd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2963a3f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a3ec7bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e258f6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@375062b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3980bbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fb7076b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dabbb05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@595a19f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141259,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1adc3c85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d47ecea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d840dd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34985e4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f5b744e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d49f1f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@603ea923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16a2f156 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24f1ec8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@303ba11b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d9618e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24701c22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e33ff3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33edae9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d566c7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7963faf8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38e800c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@662fd01c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141260,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f47529 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ea6faf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b001e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e879c12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2571edd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cd9cadf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6af02bed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58254dad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72901422 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@402da79d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b694336 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59d33bd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b1005cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62c4e7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67bddd7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27707ff8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6528a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68ded0e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6101aea0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45aa82c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@602fb2de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee83fe2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141261,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1270fa31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35dfe02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ef915f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21513ee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3465e98e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58a4bcee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ea44cb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26aafd87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d184945 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40a6d1c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ccbc4be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dfc5a65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d17afe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@148fbf91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26d3778a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59832e47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30eca903 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5976ebb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64a4b83b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241553fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e442ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141262,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fa14c61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e854ace -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50816843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@320b2af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f724ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6250dfc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@394870bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f7b08f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10849ba4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@744ed88f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ef99323 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a23e8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d4fc1a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f47e020 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d651060 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65cb0317 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46aabcf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@260c12ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@135ff05d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb19a9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3defa6c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141263,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb248e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@178392d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3544bff1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ddde36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c416d37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175f7174 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774ef59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c43adef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bfc5385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6744475f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47ea72c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c7830e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fce9028 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a755431 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e1d0f8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b83cb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5facb3dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f38a0c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@94fbdc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c575df6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76df2dfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141264,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cbb956c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58ee848f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e20ca89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@211f924a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ff73f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d995fbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5545a604 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@815f324 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225f6125 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@158cbe6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d015f5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a2a0d09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13a50b20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@632aa0ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b6e87b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6effa1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a90b47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1332fac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20f42de8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490e5da0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@249084f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141265,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2070a670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ea44fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab2ee3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a2576b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f4314a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7d4629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6133aeab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b2e4ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7753287b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711964b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@da3dac8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad1c20f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42f2fde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea9d739 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4f3fbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@496a51a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34c6d68e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44a6cf30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c0aad5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141266,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b3c0e2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9ccdb61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be15a27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e62c18c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4ff10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a3c2b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b7d7418 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@636bb85a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41f5f758 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a1a7087 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a371068 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a0c61c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a81aee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73866578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71308f30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7322372c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14334939 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54de4362 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21620fc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@431bd791 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141267,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4def244a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b8cabc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a21854e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@471b66c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7362d719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dbe8028 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b636124 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37d20fca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52eddc36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5ba983 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b01898d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c8a7778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ba1eb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@762bb696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1101043f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e2f90d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec04820 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f4d468 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c4d1815 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e16a25c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42bb76f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141268,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61cfad5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15368cc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e69d879 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577c0b84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b45f173 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a342c5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a1e9b71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d098d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a33126 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@723eac90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce574b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d1179d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d9b4d54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a46305b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d41b77a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be88af2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c9e8b36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b1aa16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17db27fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38cb80ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20946bfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a7f94e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6455ba99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141269,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@335ea43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@553b9675 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf36f39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b23358 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cf9b8ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f8152b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f5320a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bcf3ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22939b37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37579303 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c499281 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183eb3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@264ad9ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d0e99ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dd3cb14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@409f2c01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@306a005c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69702c16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ac8d147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638e113b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38695f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141270,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bfe190c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7720bfdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64346fcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@532a7f9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48872a8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aabaa14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da6c0bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df08618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c520072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b4ef0e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2026661a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@591fe440 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d23e81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59873f1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a828481 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c994475 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@386b4951 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58f4e696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68cd785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@201081cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4253655f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141271,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3decfbd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dbc3312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7906cf81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20a63463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ace51f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4656d919 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e37c95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18f754e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10becfd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@605042cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72459d84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50b7a74e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@369132c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@543548e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ac2465f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67220d9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589a1cb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26962d0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141272,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3efe673f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4947c383 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b011b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d9bacf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b53e8b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7f6b86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d038a60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@369742cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32050ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b1747e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14d0ba22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@185d035f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21550a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c29d61d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d396810 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d0ec313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65b6326f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cd89fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71c218fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a6f5b8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e1e2de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141273,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75f7e583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14933e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24909c1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b8edc58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11ff357 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c719e7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ca19392 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11f0f642 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f8b2a60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f89db93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f14f8f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@594055b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6faeacd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5ab7f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26611daa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd7092e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6705da57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cfb1462 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5259da1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17023706 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e9c90d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a60d766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66755b3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141274,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6049faf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a69e427 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26ced3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a2ef083 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@459f3161 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c2ad29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c37d51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d743f22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21dc11d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64324b23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f008b01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@433136ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@222ddef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@743fe5db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f9c30de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d0f197 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51099167 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a93b7a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e776ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@767975ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be650af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c8fc4c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@235087af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141275,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e39426e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47030a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15f84fee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32b2df01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664cc10b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3941f596 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e115a6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4affca7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3b0eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c60af22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@184650e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cdeed82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e021811 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e2a94c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d09281b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36606ac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@180c2e88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4af64c0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e9bfcb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1885ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22198868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a77260b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb867c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141276,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7490a0bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aa42c8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@790006ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51aea863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@204a6fb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b31ef71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627dafdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b4ee41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35f026c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21ae1d69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ea24dc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f0578b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf789ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a6386f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1613320f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@723730a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35cadd0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8c90a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c24003a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475dc366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241c4c35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61b92125 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141277,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4db0c8f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64df64db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22e4e191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7c4ce1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c0a25e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e26ff01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4b653d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@333831f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772eb79a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e198089 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e7b6cfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c3e617d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e2e8907 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce15183 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7938e659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6612558a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6e8efe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab5a5b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74019898 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@85a5a1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fa5bec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141278,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17183ef5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ee69e14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1436b77c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a7508d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e333a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a720b13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5ece76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3791dea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4667ac83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c270dcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a74b43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38fab141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43020169 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57df25b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c198158 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1844523 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@525e1533 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c025076 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb11def -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fbfe607 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dea834e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3385e965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@670fc49f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141279,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@678099a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e7996ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@722b9481 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b199d67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8376c02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a5b7528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a637e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23af2433 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10cb1b38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c8ce3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36fe30e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6caa04b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2470d0ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee4352 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44504ab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78631941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a374023 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bb27ce4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6df19ed7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d383b39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d6e44e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550350f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34656ba1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141280,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c86fe7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671e5577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28550db4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f2437c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cfea532 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19deec37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19e67a15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3caa33f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4504c664 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b5535a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@149d40a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c916256 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f45ae46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d3f5a04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4299262d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@257d6ffa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@768d28d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70f0571e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b53af63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@351f0d4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74225dd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618bce5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@605623f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141281,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d6e37d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bc46331 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f80138 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5960ea66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ecce6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d59f80b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25e0c559 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8d2859 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@187116ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a4a2c4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a3fc52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b7ec0df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b28511c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70eb2e62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@762c1d41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b295363 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52d043ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be1efad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d312385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cc2c70b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cdf0423 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b71092 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cfe270a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141282,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56be5f01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52c4b7e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62c71ea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f1cb386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bfebd33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c72325a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769a1371 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c001ea7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b92131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f643bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59c3bed8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355a6661 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee17952 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60850f9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24f4582d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@427b4bbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77658148 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f54a1e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3967b31f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d92924 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aed8396 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60ac36c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d78e83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141283,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@caf0b34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a48c804 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b53f7a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43a8c1d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@372a943c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b7e7235 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f142e4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab7e409 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@95f0617 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@112822d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f0008f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ac238b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bdce840 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bb26352 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74aa8978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb630ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73b44bb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ba40e65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b65af20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@358ac3f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6432e728 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f8e0aed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d4a893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141284,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aa2671a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59dacec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26fc7e75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a00ca09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a3fee4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c808a97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ca5ca4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75568916 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19ee1d86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6142f81e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a06a7ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b598181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2232a741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54c62911 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc0cdb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c121cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bada6ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e468269 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5386655c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f0e87f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a0dc7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13ff40ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73336b4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141285,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285953db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@370511af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e14cf83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36958630 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54b0ccb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2737e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@333791d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29092582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8ea94d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e419f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@652b016 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6617b06b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68dda925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415f4e16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@529e942b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@951e2fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cc0f675 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37542605 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11cdc308 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@170dbdf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c75823a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a07fbbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141286,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75adbc74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fb57d0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fe14a92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a8e94d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1258d7eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36a195b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@321ec2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df7d09f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c4b745 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1372b3d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fbe7516 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c0e3d56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50fcd70d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56961af8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aecc3f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1725ad7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21575429 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47f0cb21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5f2b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@212b236c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a46dd7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7baeb0d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19cafacd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141287,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61028adb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da306ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638533d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dd7a1f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5876b2c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d15f9a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c4042f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770cc939 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a01b58e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18dbb7a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f793de9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e2d623e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@591713b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c34c18c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ab42196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f2d8e6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4682aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c4124f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd725d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c4a9ea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66630c59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32e12cb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54fe7699 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141288,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c890d70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6276cb7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59c9ff2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ea41030 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a126019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e6b730c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68cdd360 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36366bd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f1dc7ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a4b3652 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c1bb6a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29767851 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4837ea66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bbdbc94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ae2e3bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ccbe40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@275f0ae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d05b86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cf2a10a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f9346a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ae0d92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355172aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141289,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c927ffb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c984067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aec14fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5496ef2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b642fe0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5561d593 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa05808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cac334d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565d0017 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dcb9a02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5076f3ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d1d94b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337e70bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62b4e758 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b8bb0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64e10f06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24da60ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6111cd49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b16aae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dc4d8a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aec06bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5578ef2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fb61b36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141290,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14dcc5b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4edff870 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4c6701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13eca815 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61ef0694 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c26ea1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f897fd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d3de33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11b03b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43e571fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67b6d992 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6093a111 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0e4550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58cb38e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54a780e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29413848 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36fadcf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568a1967 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4203f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ee1ef0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e9c9557 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@785beac2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a82ba8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141291,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a017c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d8c0446 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a26f8cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d447948 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a1d3d7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@178ea24e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa3c0f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14636cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57564de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a80995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71c8eb90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4234b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b9e57c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e0f0181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582de3bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a5fbf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eecbaa7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4879bbf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f7ea97d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671ca70b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c1e00cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59bd9697 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141292,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5382f99c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a43818 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d905289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ac6a82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c262412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb2b976 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9b873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289a2af2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a19d47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb1ef8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3275ca49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b41ab4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52134cc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@212d0484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d16b56a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b4b6a54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33f6649c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a364f42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7283088b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@523ac545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5268991a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a6c904a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@704114dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141293,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d5e77c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e310b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf33263 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@646c32d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fd9c723 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45c50ed5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40a05c24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45df247f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@344e1338 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@591000c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b9326b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27d30569 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63401578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0f0fc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@382b58dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eceafc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64e4d4ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2657a6ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44d8ff4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24034627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6680b7bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d7b054c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141294,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eda8706 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d6d484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2308be6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc8e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c34cb48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a3058be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cad7658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e425255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5426ac5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c8bebcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4353b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175f7274 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76fa789d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25fc2805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@646329c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8c32be8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5894fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a02d53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1470349c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57dd191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141295,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c6b52bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c930d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11082005 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a129aec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5ac1bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d80774 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51000742 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2b2274 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25666b33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64442b4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35af1a91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53f1a6d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d8606c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33dd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d35c4d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132c2c91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1571f598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c05b75b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f75f954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4f188f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@788f38b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2616234e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6669c181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141296,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38736a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43d399f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5448de94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c857855 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@460d7b48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@194e6e7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@be0afb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c703941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@718acebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bdd5b40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@154ebaec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48a92335 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff1ebb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e87ba57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7efefab1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcfded0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f8306e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf54245 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d70c18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ef7d2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33dcd957 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252d5203 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28972d72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141297,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675e3351 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@329d588c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d6fa996 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c02e7f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ad0859e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2251fb70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2272c5b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b4bb48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27362995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a11cc78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3efaffe2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e87d8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54849ec7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5072fa23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@391e1a20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@389b1480 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4819d7f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7d9f30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@467b5be5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@262478be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20049dcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43e312e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fdbdd0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141298,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a3fbfc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e51bbaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fec692 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c78aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@232a2e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70f54d47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ccdf9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64a26e66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16890da7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ccce544 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2919fb77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ebf716c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b4888a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d6c1cf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68fd3025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@458c92f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a6e4f21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c8752d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b511e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@270f18dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f405c5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b653f82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141299,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15f14b5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ae770cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37b84747 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b3f71d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a87f34e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b178dcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e606991 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4517a5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@149e418 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7647482f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b568b97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@445a8320 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7dd697 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4637f1c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c2d0a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2991173e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d3cb5f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34fccc5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b94759e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a1db935 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1226209e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@406a90aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78ea35bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141300,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f69ff85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c680826 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@181326c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@317da7aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5188f3d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dedf8f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cc9e0aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d5446e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565501e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248d19e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77927169 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d0ebdc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63aaa5c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f24f257 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bd9d7e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23519bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1563e6a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ddd1813 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18beb284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371fc484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d436233 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48abf808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@256c590b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141301,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c9e82c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aea379c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d16fd71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af553d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2656af3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a8ebf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73bf70c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43648ef0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a7f569d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3456f76d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce7907a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b139a18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bc8e459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48a69fd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4842a2fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f4632b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0e52b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f04a282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220cbadc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c61efa9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@186cc8b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4226698e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bbe14c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16345af5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141302,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@686331d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19785f4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43137ca9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c465cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d83755 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d01a2c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f7bff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51919493 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b4636b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4673c4cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38fdaf7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@651581ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@500f190d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@802fa6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dd13a45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14024279 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6df685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2323a2f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7774b7c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@716322d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22692bae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c63110e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7b5fd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141303,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2666c92d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4db9a214 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d631cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc76af0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4df8ffc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d514d6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35f15d5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@538721fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@598be295 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ef6e6ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@215263e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b303ee0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d29f259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64c1bc96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@563eb6a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cb355b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0ac74c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7746fd17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54eedee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f64750 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a0ecae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6022f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78dbb6c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141304,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1877b249 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74420a40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2feee1ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a26070a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f6619fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b12165b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13296d14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290d262c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@363b7b01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc00353 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d99232 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75a5fee3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@182b7025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@751c22e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c2d667b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a5d3fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15cc3b52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1535061 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@662ae08e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ee8cdbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8a739f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7141f6f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40de46c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141305,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1803ce74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@212c4060 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa4ecfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7e5187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b885232 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28643063 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505fea9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fb27b7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6682a97c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fec6ae7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40486e97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cfda44f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f4737e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6370c3b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@604f2597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11e12a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47cfcfcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2247b989 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3d43cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2394b7e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a9c95cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@508fce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54fad497 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141306,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52575a44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c1b8f3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e36b997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cea86aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4076409 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@741569c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f348cdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17538046 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28cb9989 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77cfee71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c16de38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774c204c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33fe8fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b8b228d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4126cada -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57fedcf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfaaef0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b77e48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e300bbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f885fc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51fbe9e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13fc98a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d1f0c9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141307,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70840690 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ebe1dc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@970a327 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c6e054f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8e3ed4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32d034d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e35c036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@226deec2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f581cc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9dc34a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cb98c9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f6034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efefae8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ccbfbee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c96519c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6602ecfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f727368 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c25ebe3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52114f4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23051cce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f40c2f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c695a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4da35d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141308,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f32baec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ec8695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c31d77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4562f627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f1dd1bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d272ce5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3507b693 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6706bdda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@562718be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52e3469b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad8d17a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8624f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@666e54cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef11104 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32be46a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@283aa19f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce64860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a3917c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a72d161 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2739a5e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29125c26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b478192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f2be42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141309,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d8e53b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66c6d743 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214a8118 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3683bdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cd4ab9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cf81d8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@94f114b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b3b656c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@423a97e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e92ee5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fa66d8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f266911 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9b56099 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60bc8ffd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cca546b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7088d002 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@529bc722 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565c79eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583b0647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e05d582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62abbdb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@570d1328 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57043eb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@176d4817 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141310,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28940948 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17bfdb21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61b896d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce85708 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143ed394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7db174e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589a23d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6146fd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d4aef0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1234d8f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b39b583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e760bf6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f9491e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bd31d83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d102cd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1337bd38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48770519 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b9b3691 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@771c7a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2901640a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59aee3fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc379e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141311,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1206f9f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3085fe8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e1a5a71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e160c0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b7aaf56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2414572b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6811580a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c95773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a750e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0ce7d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3f2b10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40c19f2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5273ef98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b88a87a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e5a77b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62ae8d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f6cb175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7179079a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6309f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a37ffdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509a12f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69e69aa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dde32ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141312,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5832fb87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583f4fd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1bda0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9943e2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46351259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6456bcad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce2313a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@442f0fcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a667ac9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0408fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6928d47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3affbb2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23d62e89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78ce81e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a24c30c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d453bb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f75faf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33b89541 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fc81c99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b54105d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be6dfb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c834d14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2310a6c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141313,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@789d8c0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255be7d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2336f584 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b639c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11df2881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc0183b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fce5a12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b822c3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f18eee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489b4b6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450896a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f31b9ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3018c21c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@224a0767 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23762818 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@311a2b75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47640768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356741dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27439980 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e25f9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20da5f4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fbc832b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@140880c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141314,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c32375 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33913c37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@787f44b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a1edd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7021ab3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aeb418a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4833fe8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5b7d0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12de4b9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@229af5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b9f61a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61be728a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c4f91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec04bb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d37d56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@466b6f64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a880ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f9d223 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@268f2b87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b82707d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5acca08a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d03193a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228fd60e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141315,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aa64bba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d17294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@507e67dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c1de47a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34de95d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aa51405 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c6ca03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a9bd575 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aabe9ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@280cc1c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37058017 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2321dbff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76129881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@385e6d7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23c05759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e939efd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca7d93e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27597452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@682957f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14749724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c8d2bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2adfd5d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141316,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3864949b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@819f904 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8c49f80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c61ba0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@403d3d54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b2d8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bbf8a74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5fecce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a1b94c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@db8b9eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a036e02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f3fd010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18a7962 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49b5ef59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@807eb13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29991553 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8a286b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da3df64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a4bf0b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b195b27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@617afb5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622150eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@171ae26f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141317,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16db53c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20cde24f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@480348b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@84ad240 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44438592 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd3c8cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59c76fe0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7416e4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f761425 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a763ab4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28acf475 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2493ec14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bab139f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@307323c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fb376e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@447bd64d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@575f12b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e9b669a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1afb03c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@287768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fcd1f79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ac703ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e0d413 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141318,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b307950 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c419998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e2929df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30273a45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38552fe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7898fc23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f48453e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4634bb80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@578ff6c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@684971f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65a5a875 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aee7fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3f2010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dda3ae9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45f4cd27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34146354 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637cfa66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2d8992 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c53e6a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bca30b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe15c6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cd33d1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff686eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141319,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45a3225d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@640cb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ada4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ef0c5d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@329db39c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12bc28f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41e7306d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29d18be9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@768fcc1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@507998e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d1bb51b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58b0e792 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50d73fb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a517c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4936d48b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e53648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f2691bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3efcdacd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4111d315 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ef9ea09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35192700 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d6bb9dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22ad461f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141320,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b549e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f685db0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ffdacd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17288486 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eeb3dc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3436073 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@316a97af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@623539b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15470510 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d1faf33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@682d86d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@239ecadc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e5c243a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4a64f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5566ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f489b56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405a1771 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d9abe43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca9f72c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@575059c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47213dc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f71fbc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c927394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141321,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@595985bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17d86c6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@678722a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7896f313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6df7d5e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39186730 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67d4f0b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28159937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9e393f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@145f09b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d4e95e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450c1c72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd1822d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61be7264 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6f914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220c8a12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618fb494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7330d793 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@438e8eb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@514d0c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a3efe3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cda3f0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b968766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141322,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a7d60d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7efdf20d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4362a8be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ca0ba03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6af11624 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8deced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dfc9c7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a54b64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66073de2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72f94f4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b2f0222 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ff9048 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ac85738 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7273da3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310ee843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589a53fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5063be46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cad3445 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a279f63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70a8165e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3df41783 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@682e0364 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb44585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141323,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e1eaceb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cbe8a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b66ee9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17f2181e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6097ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2032ce1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0ce6bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb93dca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@739e9c2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@456ae677 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e38c969 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c2de609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd57055 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@791e7a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@110597cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@129b364a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@520cbb94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c82599 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65ce7ab2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e2c3927 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@194be76d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ed30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@163b0c6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141324,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5911fa8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f270e4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284f3ff6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66591eed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304a552b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1738601b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d94d85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68d98dc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ad86ef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eb4ff00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@520d3436 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@761e3c17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ff83382 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46cd8782 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ddaa7e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aa3c2c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4398a34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee2fca6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62fe37c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62fe8cb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aded2f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13600cc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aad39d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c20a32a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141325,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2035bf17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3320b1cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c075cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4900b90a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c06a3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72d815a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41855819 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d8171ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ad74bfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78100231 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@133ec8ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15b7f436 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65cfe5e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1db672e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a802f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52144f90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@486e48de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66611a5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e6603db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efeaf48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@146c2af1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a3151ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1427edaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141326,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff89489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab61425 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a788cae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619962f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aceee90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60d3c8d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ec02598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e47464 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5943b806 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27287eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ab732e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f0b690b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b4c7764 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff56975 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cd459a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68632806 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@609c9539 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4486f69e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a218ec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf3aa9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fc2c381 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c83f6e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141327,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a222836 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c13ec9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d4fa53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668330f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d7bc788 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8f9b88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c2bf068 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23506f9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34bbfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26523ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e2b3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2142b03a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48c14a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f5f17e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f38c6e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bcf7e5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f8266ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5835b09f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41236ab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@518ed150 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4da57ad9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38619aba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62cd21da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45ee7882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141328,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@237bb154 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@177b5bcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cb36b0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aaf2a73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5420cf63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1948632f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e24881a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d8bc51f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f82401a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450909de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f572d3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce7acf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6168a3e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca201e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2adaf785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8b762f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f58a506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23894a4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e12f766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e88d885 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@646232c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dd0d724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4430a83f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141329,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c440ccc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@945ed89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2277272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43175359 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fe9159b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39e68a4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bcbe6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@791fcd29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8d4a5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b4311d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5efa76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aaf1671 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb46cd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c0a9ec0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368d5e7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3493da43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@295cf2ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c8958a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dea5578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3701d544 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79040251 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c65ac3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@336b775f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141330,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a792f83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5336a217 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@263e013 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6adaae8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299c8160 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fdca85e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5222aace -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6db3d8d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2069c384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@add018f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4deeaec0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e6892ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50731344 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3240a1bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ae0919a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70e58dca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@384449a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f3e5254 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c6e68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0b2854 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47986b15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a492d40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58043b4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c49e193 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141331,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@588df9cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b5decb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527a74d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ac5272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76588012 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a4ca05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3379dc4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10652795 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220f6d9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ef7f3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b896e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e9a14d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bd3693e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6688fdee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66f04747 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ac611d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6906798f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48c08ecc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cee32ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f00ae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce0c3b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a820b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dbeb9b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141332,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b8f37c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8b869 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55e6688f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d3d188c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbe31ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b01b384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc5107b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78298c37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46a6c3b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40bdc57a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e53de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31c22f7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@410989f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb112be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18e60c35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ff49ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3534114a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5277d1b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ad8f86f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38286cb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d0394bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41152afe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb3b933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141333,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7850ff0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3692d5d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e384b25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f230bf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1076696b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30ea1afb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c970ea0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b4cec49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28d2e17a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@480f0b59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cf9eef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30bb6f84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f5d81d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f3816a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c25c1e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47095ca9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24b946d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bcb81b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@596351df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5482b24e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a63e9c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3da23135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8c62f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141334,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f5dbfaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad56461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3652d597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bd48270 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7926d84f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48615d66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68bfc790 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67770ff5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285c47f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a94b5f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c816c71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2356ccb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@697f5cca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@260b6f04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@519e7d50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44b80084 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57328dd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2abcab78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8ea2782 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c52ba3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f4dc241 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6547bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59eab602 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141335,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40f04c66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618ed1f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b7c1ae6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47aaff2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fb18c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12ed40c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@396dd356 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292e00c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@744e202e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c5dad18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b7d38a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c0300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7461c2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@669254b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5418e8ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ee754e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1feb0d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@753d5391 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66f09b60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f1c6f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57fdda23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48c3ba96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a8a0375 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141336,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@761aebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e7440b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b365ae0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11dd6d58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc32d8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e8f90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e1624cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@245b35f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26f6f384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5557bac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a88dcfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4701caa0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@567a6b17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e13628f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4280368d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b1db0b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e3a4010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b985c42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@686f23e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e11be50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f75542 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be46912 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a852dbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141337,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2965f581 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d0e8aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5964a57b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a8f833d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c40ac10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a939a8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6620bbde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40104a4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19e95e72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fa8d00f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c35666f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e35018f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7cd2f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@340d5ffd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63dc410d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75731d80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e1382a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20bfd829 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@216a311a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c18a7aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7c2202 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@705cc974 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141338,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e206d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@464159dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c03f09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@265cbb71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f91fdec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73d2dc7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22bb9d7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@617f2d6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5131c234 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e5fd6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a4bdeeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52c7beb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d38e6bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16ad7ce4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@205545c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dd860bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@271f5e81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa85cda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72a74467 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cd41f14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@162a959 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19346b23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5178901e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141339,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da8199a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39945517 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3434c75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16aec463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ba77be5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6000b56b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b97dfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d43372 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@596c1bfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca24655 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ec6aa96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3631579f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f3b094c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21056610 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2ceabc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@261ddbc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@713bc72f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b041afe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6806628 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e8b26a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@612692e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aaeb7d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9a3788 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141340,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2833b8ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d578680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3288a6d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c40f178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6901dedf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@282336f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fea9b3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fa6a89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@636aabef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52cfdd85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f78792 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e0e1007 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388de7fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1402369b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71acf89e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fdc4084 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5839c4b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0a97c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bf4ecf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c15d6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304b0153 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3976240d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5509f480 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141341,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@798fe179 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11a62682 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f82ef7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34f64bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54dd985 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@257f7d98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da87056 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a81db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ed0f108 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@301dd2f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e893e62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f6c4231 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36dda065 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68895623 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f340f7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1274ad63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49f1e9c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6beb8693 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64ab2b6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53839d2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@157ad175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22120257 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f5fa529 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141342,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50fa2113 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1348a300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21fa1b84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ee947f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@429dc7a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@149b3c58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65cc2374 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52360ede -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e3a33b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@338eb9d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@375dfb52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@557f5399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a77650a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf13389 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@642ec82e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4be2aca9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7cdfea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f24c857 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dd91825 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73806ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fef6cc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@690691c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63ed51db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141343,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@136b4d83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77068a74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72dbd501 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b313108 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32ff7cb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e905b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d36a558 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ffd7687 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@662ec57c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd3c0cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a7b20a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73aab337 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63bf9d1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22abec09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28962db4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e83009a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@114b0314 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7d6e90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@759f0526 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638e4cdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a94b019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d9a5235 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141344,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2021d495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31d88cfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e43e6a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b700a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f1cf178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a561f3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b819c31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ca11de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ad5afce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6050fe5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57078671 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d129ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@229d4505 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d8ed06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e87cfc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62571fe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a2adbc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2898b359 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f19057 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6203189b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e8b13ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7364790c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47232c1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d1e23a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141345,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bcccb17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b30717b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752de3d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3962cdf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24eb2db8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf83bf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16d14aed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f3188a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@440864e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@962d814 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15f86df7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ee008a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3346559a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c7b899e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7abade33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e947bbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f7f1373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f385a7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635f5ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8b90b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a8e09e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ce8e52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dff2634 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141346,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f530514 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7181a870 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31fd140c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e75a81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@728b988c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eb56c85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5731442b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8dd3f1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@263c8971 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3480877 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e050429 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fc1e538 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df484b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ebc20c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ee5f7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b68b7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@213e51d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7edb875f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44741a78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@855bcde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b369494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797511d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1db6635e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141347,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38a09f7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@575d2ece -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6774e76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28757175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1492704e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a87074c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@163ff894 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c2cdff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1255cf09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fa1768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6968b4bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a46215c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d7969a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6468a7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aa73b60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7130a3a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb98853 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@291efc29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af1be51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e286f4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19848dcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a5c566 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7091e8f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141348,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dcea3ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4019830a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@297ee09a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d77300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e8c61bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5281a8b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c25f84e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6187c5b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cbbd9b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a01e9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2064612d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f11d295 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de5d2ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e327f7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ddc149d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca8f4c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f0b87e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@322fd2fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f4a50d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@640fe745 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67b87887 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b4ce6b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56fda79f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141349,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dd347ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698b62b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@253e586f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ec4c208 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@497e3b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b52764f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c78b46a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1494e76f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c8a10a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4402eb73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d30cc7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e2cf92b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62dfed45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3155cb1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aadaa6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b1edb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e998144 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618bb8b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1602882c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ab22b23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48931690 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74e16918 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a0dc40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141350,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da01a84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dd7a836 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3281a13d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@423329f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f5a6f09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3ceacc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@100b3e5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bace10e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4702fd55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f2015f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46902b8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a9816c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a20b690 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a0a3aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72ab22cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58af0c53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6784a5e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5789f749 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c1f3a09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cf26c4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e9b6996 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b0f6b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fa5b1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca5cf28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141351,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@937d8f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@833a208 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59c03abb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bfe6224 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c39c5e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@190f1585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4142f1fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e544b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bb1c6e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af3412c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ce89096 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f8b9364 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a48c8a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a0210e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12d64913 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11c5e89e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9427c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ab93a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5795c29f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@692a51eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ce916 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef11cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5789a2bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141352,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c1714a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed17465 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28607e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32668dae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66806d1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63774ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@484fa69c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4259c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bcf258d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@729edff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47d0ec7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b000128 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49b1ac88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d54bb35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d2715e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16355c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52246789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ac1e3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7c5970 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@331498b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42fda974 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7796d47b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141353,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a4cb420 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d0a7ec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2293d01d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6367fd38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf5a8c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d721e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ef81296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@222abe4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1328b161 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@688b0284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d9c1a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21f6cde8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7703c30e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47ac6048 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe065d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@486e7a4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@219cdbfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59641a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5904420d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6283ef7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25879bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12ab7d82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141354,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69c451f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14418384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d00321b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@448e0954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f0e9be4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17cb7422 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd70c2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f2772ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@540d0d16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@182b09d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bbee0a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f984f62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13c1823a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e47412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f93253c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@721d9af3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5216d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1de6ab84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bdad7c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e0c4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78173ef4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@508500f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dacb1f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141355,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68350a5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29147931 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ee2622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29b516e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69128668 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63c5a78a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43ba42e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b9466e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f65b997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d2b006 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606e61a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68ecd1fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a577476 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ee1a9ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d91f8ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fcb399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c99354 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f64ece -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a2829f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a48309 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bf9f5da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@581a5c55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f6b8bb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141356,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e46ff9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a659cab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70978f46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33e2b4f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ae1070b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb74238 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10160b57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f548b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@342ec83f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52eeea00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72821fb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eae41bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c830366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@918421c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b05fb51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@773ae42c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4316419e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@855f342 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c4e4e6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf5d84b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d9e6efa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a7651e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47565d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141357,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4600c297 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28878d13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@207670fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f02cad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd6b5de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bad4622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@484f0c0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51821385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b00242e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a2dfb3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab2b316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bed83fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4166ebf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e73cc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf2134d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c8793be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac898f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c49a75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c3797bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9de3e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622ccfbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e8e6bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61b23574 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52035841 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141358,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56563489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@207dedf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5094751e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@181dee1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f0863eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30550397 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@439aac06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3400cf6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8abf53e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10fa0607 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6966282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345fb681 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630ab731 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23004f69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b7abf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f95ec10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69106bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ca1aa0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63098823 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@130c0d3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79a94bb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@612e5137 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec985a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141359,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ddf04bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e986c33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d2c233e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f4a49b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5af57c39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17c16aa0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b888eb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26427582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5727c1c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4baf25e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@647de70f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51267067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38e6aa7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ad450e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a8b2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635d6633 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e659d67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@469eab36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13889339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b6c1385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59203b2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a54fc23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141360,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d92bb0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46196707 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e38ed85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5340bf85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b220c05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@760e6d39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f83e2b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79e9cc5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1240b546 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a27592 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2704d3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1038505a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c53d9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34fd611 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2556eb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503d5427 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e3aaa73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33814fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab1e2fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@101707e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b554ea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141361,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78090afe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d8aa908 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@758a8d5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64bbf51a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d55d89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c1f7cf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ddeea52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4676d3d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38b1e784 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1feef48c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e81159 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65709851 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@524adebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ae81aed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fdfa630 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e8c098a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ab799d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70f2d510 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@628e1318 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca0969a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a6fbf86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141362,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cd3a78e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5769df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@464b22c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11f2609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40311e73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355d8e3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9faf33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5248eb07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac2b5bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43b7dc8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@148615fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@103d4b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee3c64e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73cd38b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c0eac0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48f95294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58382c73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@422a877c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12ece313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10836e28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9c193ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f46f4dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141363,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a47785c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@613eb912 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64e580a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c2d68f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d796cb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@91bb020 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4133c92a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66fe6b39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3034a79e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64fa8b08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5f6424 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eed77d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b703dc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b62536c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b5c7860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ecaed01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ee472c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d8e9bee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5397947 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc23e04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad3c597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43db6cc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@de6c6b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141364,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4099eb13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@446261ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc7f882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672f4ec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6861f044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f5bb193 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3b4d12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@417f4912 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@267b62a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ce98dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@429b1e8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7997f812 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@404b3a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@343cd387 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec9e4c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6858de67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4960f164 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b92b324 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a1c9b56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5499d994 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a3ce9b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14b97242 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141365,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ae18da8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@102ea019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da2bad7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32d6fe43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345cb97c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5197f986 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299b9561 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3a42f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d7c8955 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134904b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@660cbb20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18df70ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f06483 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a47a4a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5911d177 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aaa92a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38fc2d03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f79cc3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@83424cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e5c0b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fa87d52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ea14eb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141366,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a15e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f32e5b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@330c43e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17fc67c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@401ed5d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18d9ef4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499bacf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dad1063 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@babb21b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c5f97b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48b764de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dcc5ba3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b21196b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be3d59e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1801dda8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67ef8c48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24463b9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@517c45ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47862eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9077c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499f250c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7afd48b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf83ac1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141367,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530972fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638e9ad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7326c6ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ec5c47b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c3f7f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310cb9fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d80890 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@642eec6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5debeb7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@97c425b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dd4576c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0a9136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb83f80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@473dded6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58cd8e6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22b43764 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f9d488d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ffe4c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f598ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51798a80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74056902 -17-08-2023 17:39:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141368,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc56a90 -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@130ba775 -17-08-2023 17:39:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3707217359314889721 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e3b121e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512937a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17fe56fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59007b78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ffd39a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17f18bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72dd6e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285c10fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e8a2c4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@553ef139 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c89238b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a3929 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bed95a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f818de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c68e111 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@696e54c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141369,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e5994c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671e413a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c8544a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@756eb81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@557ba800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40f92404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76efd0f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d53749 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e66ac3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47b668f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd67897 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2056715b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5baa061e -17-08-2023 17:39:01 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f2f3c4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5341fbfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@400e9464 -17-08-2023 17:39:01 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b1ff08d -17-08-2023 17:39:01 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0, existing:[] -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d0478fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4146eeab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141370,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75dfef53 -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3707217359314889721_0 - service started -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3707217359314889721, topicPartition: test-0 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4170f2fd -17-08-2023 17:39:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f8da1a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9483b49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e7e844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@454615df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5754145b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f8eef8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@231791f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568a80a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@210ba0e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4911f1fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56964108 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@549b74cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@286bc21f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e3340f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141371,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a49c239 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57beaa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7376ab22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736614a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@432a339f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19debff8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f37385a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bbf3ec7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@791b5ee4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15561f40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73451058 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@611e51f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d1cf48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6758744d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cc46f2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc6209e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a111a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47fffb84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@585d414a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4601480f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141372,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d467577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@155d8880 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2738cc70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b35b9a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c69474 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c260bc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d9c0d7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ba2019e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8645aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56420b15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b381b4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@602d5f44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@385eedfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e024343 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65fbc5b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f267130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20584037 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cac1724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7741ad60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b32fd49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141373,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490949e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@653072d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f60e9b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@238497a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20418c70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4681cc48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e34c800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@598a35b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@758410fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d2f7189 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4523cae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d0c5045 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@185aa2e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141374,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ebf3b5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@590ae83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a31580c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67b1ba01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40732846 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4017727d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6046a34b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a4d7cf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a554e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@285a5074 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d53e2cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c98ab69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6563d671 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fc0f359 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de72ff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@247d00c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd68a62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4794e0fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@348ff73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fff94c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@249192cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e23597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141375,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15f91f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337f6401 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd09f4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4b5f2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61b99afc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29aeb5a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@361b4d1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@223dfef0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7efe31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a169fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f3ea523 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b9064a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a4ee15a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f89f6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18a52fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2214e1a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fad808f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49b86ff2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78d1e134 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dcd3659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e577897 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@441a5950 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33b5c8a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141376,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@275e4f77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b808cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fe1b15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59931bd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce3d266 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0f6d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9deb06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b91183a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797387cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf299e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f284a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a410769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f69aa0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57dc35a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a46545a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@280e7234 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a25a4c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c62b75c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@242ec60e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f6a8f6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68fe1fc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71bf9ddd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f32b52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141377,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@849c9a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10d19f31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d8da983 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@271ea8db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@615c4167 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d68626a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d590df5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f5324f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c070101 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62c243d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2731ccd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6183304e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e82a76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c46d99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3a70b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24c32a3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60bc0f49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36421e62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f267ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141378,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d200f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49bb18a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228c1e75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@86ac1fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@655ac5a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c143f67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e29f126 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2012b237 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d63c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ea4c675 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f35a646 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a03289d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@281448f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aba4ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123e252a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eaa030 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb31b03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56e25a46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@351632c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6129ef77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72518b52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3625df34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a1432f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141379,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cdd405b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62285389 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aecec87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@426c70ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34452f40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1451865a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@944c883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@280f2365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@352d260b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5605a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@279c05b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53bccf7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f7da678 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@121bd920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e2ca820 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b5e6b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9724ef5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45ec4422 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e3a8917 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66a0ff3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d3e10fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad4962d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@708cce51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141380,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11b4ae5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c00f566 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@621b7d35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7473d961 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40815ffd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bb1394e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627a05d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8287027 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@421cb868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1035822c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22531137 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69b1f412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec9deb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@725101a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48317c77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec61007 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7a5715 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54dc0760 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@174d1b48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@915778e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ab4b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41f8a594 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1071b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c1bc8f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141381,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33c5414d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371596a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ab5bc4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@482bb1ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51fd4edb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ac642f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d36d825 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78050258 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43b51015 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b1c3b67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b03bd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30f7e25d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b2969ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134b6ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60bb6bab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd198eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f27eae6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b00b823 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60dd66db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@328bceb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@439806b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d076f42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16673987 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141382,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@506e0044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5340316d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@751b2938 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a1f9d1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a53da3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284a4744 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@434e99c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22657b72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4603f70e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c89f8ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fd28ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1665d88a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f07e6cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c3a2152 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371455fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f8d7a1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f4cd4ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12114ab0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73f668d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630691da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a767004 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622ae130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141383,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eddf8f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490232b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd9dc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@352bdf84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c09b9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c0f8633 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b05bfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8c6d90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd0111d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b8e3a4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4993e175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310be53a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8842ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@209e698e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d0aca1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189cb574 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30ab9322 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@632a4ea0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@106c43c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da663ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@178cfb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3096de78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4296f9e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141384,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf35914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b315834 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d523834 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@287ecbc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec31c24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f621130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bd1570 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc3c83a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd7db9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3453aa41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f9e2f2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c6fa8ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7981bbe3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50291730 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed0b544 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f19d431 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b332c92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@433b2053 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@80ad7e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69532289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bc6c48d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e5501e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f8e0c38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141385,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f392c2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24671ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19b8b41a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64884d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5046b046 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61be72b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dfa12fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f5627d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a776cfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@84ebeeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@654e00da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2be8d304 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ad94a09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779d145f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@624092a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f5d1d19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1497f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50345da8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368730aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d546b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a15b82a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48be7bdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47331000 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141386,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c281ea6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b6b25e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10b0c3c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f5c62c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33fd98a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@414bf714 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5773922c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b959044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24721731 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a1fb7fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7136b061 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@267619b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@120949d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55eeac39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@119fbb5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b2835c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2d05a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f02e510 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13abecfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1025b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3727605b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe529ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1365e2d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141387,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12d94889 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ef0d731 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68ad065c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54622adf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c27bdda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14f6ba30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e9c114 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51983584 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e08b00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41669a16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5d024f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b7920b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fa51251 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae68399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@695aa5f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2be45ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c94e451 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c04811 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e09109c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778360d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c7eb41a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cbef344 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8581bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141388,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@266cc5e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f7b6b53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d4912b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ba7c77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4239fa05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ac30e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b1aad5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@673931b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6add1680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de10019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46bdfa72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6827f7a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a4c8536 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f58a1ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@369fb65d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6152fdfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f39a1d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3536284c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed05de9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@89beadb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a0edd1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efef458 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141389,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52bddb7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a72ec5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@599ded59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@188e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cc2fb7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@af2b26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36eee94f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72c2392a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f03d1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20ffdb1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@611afade -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27284be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab25131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cb5e5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32319816 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2470bd9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c835bfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32e4f9b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cab36d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d96c6b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b3c3444 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e8067e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc8cd4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141390,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7033c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a54a996 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77266e40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a596f57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bb6f7a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3431f35e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70a1dff6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11632562 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@df4f326 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36645ac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48446d90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@408419cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a43af3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee562c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a6a864 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@750c817e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241da9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57e7735d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7722044d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66b46ca6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aadffe0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63c0473 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61db6895 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141391,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24cff954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@211850d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e1c71e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d4490b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@261ac312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45e799c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c777f56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ef9747 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@251dccbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4d705a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b77294a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@420bdecc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@938ba22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e40d4ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6caf8704 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@704ec9e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6b8462 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@434583c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14b36819 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12b4babd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@721c641d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@544f37da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b336eea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141392,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce54663 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dc8fb6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5014e2ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1010923c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56801803 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca98a36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5d36fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f48d6de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@595e9734 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79865e13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b1e5536 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42a98cad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fe354fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24def9d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d90bf9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53529c61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e35e7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75a0331b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d142e5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a620e0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56cebd83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@220264aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee8e84e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@540099f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141393,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7701045e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46970920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f543324 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f4ba06f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@97682c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eb7bf17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70f47ef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@521ee05f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69732438 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a7f6106 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79afa971 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c96b203 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c341f4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d923c9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79f2e67e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@109d45e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cdffec0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c5471b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eea8e6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64a23252 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8f9f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dcf8b81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@171ccf85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141394,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d95f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@223e5d0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38c4949 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@721dce5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a101a60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a16ea13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb12c19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c2502b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5b87a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a0b663 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7293918a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17b5d4fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66227f6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a4de62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d550e5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24fe0300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d2f9d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d07a45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31c2ab64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f0585f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23aa2953 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24af7d31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141395,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@926b4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6747e9a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15d5b4ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4acc05d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5917f7f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1452fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@398d8bfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1345eb88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@750b3e7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1b572c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@81cf50a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b9562a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@186d761f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f48e009 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7093e24f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e65f271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dcba02c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255b4f34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab5404a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b62d517 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b88c297 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34d5163d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5402a7fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29351a07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141396,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2424f9cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@385c778b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79a01232 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c4da23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7badb66e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@201676bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@595ec10d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7951dc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@119dd59d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2adea9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d54156e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2005e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d4e42c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10dab6e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d283ef2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d59b21f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@128e9b00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57a64215 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ba529bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bbcab9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d0c7d70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@585f262e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a951459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@444ba0dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141397,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af2a2d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bbd89fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69d8750c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@700c65ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe7ed10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2842696b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@218097a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@112a15af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e5f5f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad7c791 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28464068 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6980bfa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17b4a528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@438b1300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59545e19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35c5973e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ae94b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c62e9b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67bdd8b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed8cf18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ea85139 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258c3a18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c7d2e77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141398,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@146594ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78282ec3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c308724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797b1843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4181396f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78e659aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4057f835 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@614f597c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294d5e7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@632c79f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cc5b3f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75a8dac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44a7b5ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4444af2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752fc66c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26cb1288 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fb8698e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee49088 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35f33d9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b5226fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57bac207 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4078a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6173b353 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141399,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bcc3c71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ae76bf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74acc139 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8390dd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@524c9716 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5375e669 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c25037 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@155b276 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3797616d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bad8f58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183d2df2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3457fa81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3847696f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb7e670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e941ac9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@659c7668 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@661367a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b79971 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8bb913c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e129975 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7341f95d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9479a55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d21d830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141400,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47623bd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b95b970 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e350965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d86a0cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@becbc10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da4da24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bfc86d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5394f998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3df6d3c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b84ba6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59adb783 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d81dd17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e25382 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6243c983 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56667be7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752c7d59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41664e3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d159843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c96545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e191db0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d7da5e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3bf584 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fcb4e0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141401,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39b930df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@693b80e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1adfdf93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f8161d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f4e9932 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a471c96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e006f0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2afb32d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5187a9b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a6acb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e03f1f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d1d1f6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11bbe345 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779e4ea6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6603e595 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7370684f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d1f0caf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f18ed4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4127fb12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e5d169 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74dac98e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40697eef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee8f29b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@211912b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141402,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad39a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b8a13e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@713de698 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1858c6ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72529c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fae620e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21395a45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f0aad57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19dfd559 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ad7a7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eadf9fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e78b6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7375e1b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4db50b29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c86797a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34121552 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6836121a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d3fb076 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225d4502 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6416c9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e3f088e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef3af3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ce9ab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141403,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be63f6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dbb21e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e02e538 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c8bcb64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@364c8ccf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22ec37f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2e2db8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79cc1f52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b34ce50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68a47f39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52dddc78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bf26ffe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b96e9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@daf8cc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@312daf16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6978ab52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6438f57d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15809489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4909cc30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6587dc78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46374ea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5782ae3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ed80b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b52f2ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141404,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@776caf02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35dd34a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@909f296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15d1a5fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7571cfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b734090 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d28942a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce0409 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ac158b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b77e8b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2227ba38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@573c1c39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6708dab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@237edcc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9744ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b7d5a95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a01fc26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcc8545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797a2e42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18b27a0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d7755f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@268a19b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e2a2b1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141405,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c372f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c4f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74ad8129 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b002e62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@354acd48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ccfa65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4df09d3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a9c9de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ea78ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c7a5c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f97d75d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d91648 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1233975 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a414510 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2d563a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55d86241 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4da8011e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dc1a47e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7cb77a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aa01ade -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770d4465 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37affcae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee15a46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141406,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44edd1ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59de8b3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11dc5623 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255b1d34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f91d890 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@223b1503 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51252229 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c6a3784 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d3e3fc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57d7a468 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f1bf0d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b024a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eacc901 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b2d858 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@145f2167 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47998a61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f15b752 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d17caf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@302e9c2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3befc2ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39425dc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a614050 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6a68cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141407,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b560059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7051fabc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e25726 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d8367fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bce6df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79c768ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2e5bd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d6469db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c8ffab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e4f6b91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27067b6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3932f4e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a4bf33c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f3d1007 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b74c997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45cf95db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d85c4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1772d979 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a878271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63dfa5b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a7a7b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@201f0be9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ed9191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141408,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@588176e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4407f84c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c9fe366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36d4987d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6531ee07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0faf75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d68fb01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fed83a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c6198d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66d16a47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cdb256f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2845de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a1725c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779e990 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c444d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f17f777 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c7dc04f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4808539 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@644e07c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c2e879 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@669f2716 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6636cfb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7252f4f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21805beb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141409,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb86f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e23747b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b33b873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284ff5fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f09d11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2005ab78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36e36f2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61d48417 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@309d8529 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d4eb629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10790c9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5548c6b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@920a5f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4117a506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d24c54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a3c34d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63f276f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a7c9d3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7147decc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab6fe80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65cda6e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b467a21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b87a3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141410,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1957c0d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11a5bf88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@597a48ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee49859 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9929c07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35773b42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e412033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346d41f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@298281dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25230285 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12d4feaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c685fd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d9bbe3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd3a877 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79c72ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@775f70c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4be5db6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aa44275 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5497ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56011d79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33da927c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b55c998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7542c6bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141411,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b76e7bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd8d9df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6964d0d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@af0b45c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1627f1a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c3bc498 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53965db2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ccb51c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75481112 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37950427 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@560fa4aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@504cfab2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65b9b128 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a123584 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5113e498 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66bd1d62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e25d06c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25706239 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b89752b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65513335 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5172943 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23d0f272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@350fb0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141412,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f74b8ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5315d956 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f2f224 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@737eef5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3906ab12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44241a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2427b37f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2027b4b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c75c59e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4865566f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5140a857 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5f1ce0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e0479c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb08841 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78669f90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c6a2460 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21cd9e54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@997e8ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45c17c11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589519ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e71395 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23034b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d6c9a88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141413,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc6bcb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30906f2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9f20113 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43727f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c835ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b12ce9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f1ae73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19baf4d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b667b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5d092b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7250b2c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c028463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcaffb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a4ba537 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dfaf421 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35734a61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dec3424 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab948b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6028ea39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f27941f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3316d4cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530aeeca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d4bfb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141414,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a04a59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21576ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66db4dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5715a29e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52e2d8a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be95637 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1049d6c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab29707 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed15d16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f38ee15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4119c4f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fed697d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b70aa7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4360a746 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd1e5a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18a3dbce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e89d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@400254c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd15cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c6d3de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77e482a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@259264c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eea06ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141415,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d0174ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e233848 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1821c505 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e37cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b3ce8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@476559a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51fab5ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c611d9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2248652a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f8dc8db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bde5b5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f172a5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e976fa6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ffdb9bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76c02f91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b386e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@158c2b0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@379e5840 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d54585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c0d5ff9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68adcb24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d34d229 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a74367 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141416,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31a32efd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37d12706 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cff1e4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@413ececc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6942c4d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@571537e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@135c4622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2423b22b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b38863b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25ec48ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54f06703 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dfb255c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72ed7f86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3580c309 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@621f62c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@567a6a1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f723e19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c592475 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65497871 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e6dc55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ebaa4a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf54588 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d12e77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141417,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73835aec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18775fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@398f802d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1581b3fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cbccf9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e94758d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5f189a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d95e57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@190399a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64c6af10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bef0da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4edc2d9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b97637e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a78fb33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a30e086 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45af9091 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d387c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68af5323 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@576ba60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c649947 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a0accd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@784c311 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3711cc3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141418,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f27d53f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f71d2d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e4c1266 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27057580 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@369ad272 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c7131 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79ed8c9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75485f54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a11acd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@710034b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@649ccf61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53199e59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ef38185 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea69156 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c43a624 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b70ca3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719a4ef9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b3c4dbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aefa678 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41266b92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@626808c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b20e09c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cdb45e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141419,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed3df14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c5fa447 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70878150 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76751ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@466e5476 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37e2ad60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ba327cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@257e3dec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@110700d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33ac263c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d3f2cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5162440 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3ebdbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ef1f3c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30fa0379 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fa0502 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d70a343 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c6e14e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cef2bd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1869f55d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405b37a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c4eb21f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294a93d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e5e58bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141420,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a1c6af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e165e0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733f881d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f288d41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16d5a8b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a8f624f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25ff6863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e86bb18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25ad387d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74e1bb9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a036912 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f65931 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dd4e7a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@341080d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ea24bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3666cae2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c65c27b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c43ef11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38d8f1fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38691dd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a6d66b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3925238e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141421,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@468d4271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36fa18ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17039808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7aaf035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b18b47b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d41e6cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4689e4b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16728f08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a75ad1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a12a48f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33375b3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31bf989c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@738c2da8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c95d98a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f29f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c496cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ab71911 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@789cd844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3953e6ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51d65fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37508bdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29102b8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fd2dcd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@610e6a40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141422,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b30bdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a572e71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637a41be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71347bfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b936bf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7874ee54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49b23755 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8b763b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be74a58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64be053e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6e9e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c941d7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d202d3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69234054 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21764695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd5f805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e05b71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72328af9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e328c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33a4f12d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@301c0ffb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@569f2e43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec2995c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141423,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a7b2830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ac88b53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450a5d7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f3824fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c61b984 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32dd3b48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252b79c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8a9e6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a36b907 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ead472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f710ab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60c7b0f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f06d36a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5848c5bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcb48b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225617ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b221d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55d954f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70720830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@628b82ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@391a39de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@267b9cc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@462833b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@780a557e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141424,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10b64768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a515430 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d637754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e86e23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65c7fcb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d7d4ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73f8cad4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77e52bfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d9d63c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16d6ba25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23cd6c41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da9091d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b6cce34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18484cee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50af94c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65fb8e2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383bf1ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62cef51b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e6b7404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77583134 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108a0b90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f942b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a7dd58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141425,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c08fe9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6656d62a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2975ccad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79689c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@250bf11a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@753c9417 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@722b3eb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41404183 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bb9eb90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a345cdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14171d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@424e7427 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@365fe295 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e058a3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34408bf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e2e11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9c3f6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e5a18ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e750798 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a6fd2e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a5f196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3934a010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f3b2969 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141426,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b2704b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3167f06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fbe7b54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@396869ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac9d09f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a17728f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@469fc842 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c40c40b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6859a634 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@794b22ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ae3809 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf4dcab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e39ade7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ec70a68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e190bc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13559725 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@90563d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c2980d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4003945f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f50dfc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d734fac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141427,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f23b2ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@735e7a28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a093d71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de01db3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f66feba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe95044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c481c7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a887c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d33463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c2d3c6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505a6261 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f9793f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@658939cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22bf47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3688c48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d762d9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d3ebb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14d06148 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@531db556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1b505b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@279bd7e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3056387f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e5f9c26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b538c3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141428,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@620e444e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@639a0db0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11d60349 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e34f31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f5597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359b3f61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14f6aed0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1860f1b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64efecce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8e3b705 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c218e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3630d156 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da2d2c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ec03ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1503136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31daea2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a551a10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527e9bd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5227c92d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@344aeaf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16edf3f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30eb62c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c69bb59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141429,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f17c74e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@461cd934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d0a0e1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca8db96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d0456fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d71fd17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f67242 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a7861a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d44792 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175f8b7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e89fb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc48be4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0bb719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5164ef79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5171e7f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68d3dd92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f071cee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@796e2550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27fe1bf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51377dd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e0fb75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eac165c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39daa9eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141430,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e982f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32b3f0af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca30e53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50da7ddd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbbab10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62a00cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d55751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4562fec3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57932e54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35b7fe6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59344852 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b722151 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45dd4fe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f13ad9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@341e1af2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efcdd89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f11dd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53dc0f0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1751560e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f89c1f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@569f6860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6508a1d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78f46a50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141431,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6963e2b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44125b93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64342ca1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f497c9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49495799 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fd153e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ac5259f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e1c17ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dea780b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38316005 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bab4341 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c79f9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b93950e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4891e331 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b11029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b347f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76f8508 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@402066bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@114116ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb767f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@108bf843 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@535d6173 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a33da94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141432,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aebec8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@556d5673 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d83f354 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3930730a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c7a6f25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@439b7017 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31703263 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@be2af64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@244efa7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ccd2d2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c1ef3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1219f494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d74187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a6a0dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5c20a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@323cd48c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@491c7035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@681d898a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664ac536 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3206e5c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18b1b66a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c642d2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c3738e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141433,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299ac87c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8d39c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd7628 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14d8aec7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b7e5d08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee83285 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@548e1f46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f13d86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cfeb62d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637dc32d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47ad942b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b3c9c55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@221a04c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ff3cf04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4541a1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b25f0f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ebd55f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6851ba89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c71c16c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48573a54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f115f2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a7121b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141434,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6849ef36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@193dedf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c1039c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f4fd77b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58d8a7fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7c9b74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73ff416f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@728aa464 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff032b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@251db300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ce06af5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b6e5289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65659cef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da8b6f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0d672 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb336b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f84fca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a0f6053 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e883e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cecbdce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9a2386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d36cfbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7250b530 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141435,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0b7479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3690e48d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0096de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a0e477b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad650ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd365a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@328cae66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad102d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258f0de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e65b74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@154e22d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475b6d30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c0bb05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720d7dd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16b199e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a1f44ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d00de3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147ac1a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9e2c5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed61172 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75acd9b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37196165 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489a5f0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6899aed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141436,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15fbbadc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6995f34f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73edbb47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39790925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711c798c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3755f36b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29fc4145 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583d457e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a7f87ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5229fb63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19946e0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b30cef3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d467c9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a97eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af7b497 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@179da26c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39918e8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72060bc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d25a1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fa60808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a760a97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@786c662d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f09d186 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141437,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20d07c27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a78e818 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@313eae74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60eadc34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4378b54a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e2dad15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a24ebaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9991a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5acdbd9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19cbf6d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54ff37fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fb8863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73edcc0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d6d1cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183b7865 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4391 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d8b5a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e5088fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5deb3301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736e4e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@402a8d3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a27eabc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141438,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ed0a9c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e71772a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a264cae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f313b5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44f41fc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d9f852 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69d293fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fafbe46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b41ac65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7534cc37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@804d421 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a5b16e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cefa143 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe9839d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e2f39d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@335c5d16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52d81787 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512e932a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27a6ec89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7643b186 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6435f4e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41278c89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f5f280 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141439,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a51a6b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7108e7d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb66ebd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72a9ec59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45faf8db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73fab9da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c06e29e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71701463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635e06b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@454c576e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea55f4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f8797f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c7f4300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7baf88c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f64cfff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c2fedf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a6eca32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d009c01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7452a769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1771b45a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b02c3dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@625bac29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35326e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141440,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@199a91b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@544bad7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bce6938 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@713006a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cea1e24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35437488 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ddc4cb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ca842b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30df7c38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e96b99f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5160c738 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f438a59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4361135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc71a6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c941aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0b1da8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f800e45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8837baa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a75bf01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@496013ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@560dcef5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141441,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45c0a1b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@525a93bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1beea789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@232cb867 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a168be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e0f83a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b5cc9e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d75a76c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2579c2e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@473acf59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ac41d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aea8164 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@216f143f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30be574c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a1ccc69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24638bef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df4bdb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fb29175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63117d6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b95be10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d481ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c691d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d473a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141442,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c72096 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@113b0458 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@def17ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e452c83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6189724d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2859a3e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dfc09fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a9a17f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30ab15f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5588f6a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68869e3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22692956 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a058441 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582a4e5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4817c2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc9d670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9886044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3da9795 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c35e3c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2deece65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67ae11dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@309f4483 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141443,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1317a29e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4851cb7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f661358 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1517ff6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3abc05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df6922e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1265b21b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38a3fb9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5617b21e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb93d10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2073b196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72bf920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c472c99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a40cdbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7823463 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fb82f4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@acef3c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36dfe208 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e8f99c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3450b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4399e441 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a1f8198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ddcbd5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141444,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7171e1a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d46a822 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ba3f553 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39d9f34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9f1a0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d0ba26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214dc8a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e469780 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b0cb325 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b3d31e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347aeef5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4efa856d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6041820c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72614d9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@118f1862 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc426d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67f98445 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f0c3082 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76297709 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c7135a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778159c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@977ef94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34db508c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141445,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6152a78b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce09d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2addda49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ecb0ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30f62947 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28e6caba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c2d527d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d65e696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c58526c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f5f247 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54cdfcde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@367ee187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383b1e16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45a1a49c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e318bfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f823f62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f295d2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e49717 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f8a04a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4baffab8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e7a3eea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d15461d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4f4105 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141446,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7901182e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7603e48a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31e161f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61159a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7720f12d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6e7cf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@243a286a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@286ae155 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5833c4ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355782d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40db9561 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f44b466 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@799613c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70de1767 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c32f480 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@764f8ae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f8eafc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f6bf271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@448697e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e942d36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ae3bc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6436a95d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147f5be4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141447,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b6d29c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee8754d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fc5598c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@310b99eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@233a8523 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eb5ce14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b49f377 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7660e534 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1046546 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4101ce1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15074f00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@643f41d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46333645 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@98d61b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26dd40cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3872ac0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672c3459 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54e41e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d9a2ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64667541 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@626e1b68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@155837a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141448,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11108bc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@460b28dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54acce2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d162d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c6db6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a9444cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@387b3e00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503f7b5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7381d79e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1b7739 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af1c964 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@113fb0fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c7fa863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c5db86a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5853f66c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34071dc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25aa8790 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2435f5a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631287db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f77777f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e90eb06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1583bdbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d8dc8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141449,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c08321 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f919624 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2777a025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f14cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3184460a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1287d6a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@86b2a55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73d98ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@607e328e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c608abf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d8828e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2219437a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33e2eff5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b1fae44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278c76cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f66f3d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bdf208d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6db4ce3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294ad76f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ca51fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4383d802 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@88d3b60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778374b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141450,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74459e1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@295b0b5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ec732c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24a6a07c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ccb7f98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dbb8ab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e5fd3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37510cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c19213 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@417e4487 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48fb4d9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f39fa6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bbeda41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5857e1cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ad2e8c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b7409a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cc6fa3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b782de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63102e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d192129 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160527b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0817a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a00aef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141451,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56e473fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6963de0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@251a699d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27efe8a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52e13c0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258503fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5325c057 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd79060 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@508ddc58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d9a9769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c0d3f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1723031d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@297272bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@205a8925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a3a9a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9976982 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33ca0597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@472db16a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c58ee86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eb147f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d34110a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27385727 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c8fb198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141452,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e5cf08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75f4295f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1010c9a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5152b942 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2739a1ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7353da86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@659808c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21c4259c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b98ce92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26f2aa8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30948bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e17587 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e80d737 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d15b2a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74904905 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c55575d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74640e8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64747259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12dad7b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e3950a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a7fb6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5706a598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13258784 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141453,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b360b2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6465fda5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62dbed4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@390b0032 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@429f1b4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@95c5e6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62ec8a8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3870c72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f83a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@523cbc43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5402fc6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5d8e05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@170cf08a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e5988c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6347c95f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60695e94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10116ab3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cbcb1c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@416c9e3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b5bf57b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70845e51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9cffb1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c771966 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141454,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18b3c1be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a97622f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf66f73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28defda2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee66736 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f91fedf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50939fa9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fff4bf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123d42ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@420a2174 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ea8904b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bfdca81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32ffd06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@103d6d96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5add9d12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe1fd0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33daee3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48f0640d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e73550c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ebaee65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5af54489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11ee83d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75516fd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c716edb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141455,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cd0a0d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9457eaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7794bfaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60840035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93bfad5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b294d5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7992de7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3957fcd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f6c4479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ece8f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6140e9fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6777185e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab5d745 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@107b2116 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@769225fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29f8769a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6db3931f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24896c2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43d31094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29f5f2d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58afb798 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a848bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c24b96e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141456,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ac1b55a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff7cced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aef881c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7349a9e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14c5a9b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65248f61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23cd54b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1df5dfb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24c46d90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@655ab1ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cfb1928 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fc07367 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@186f35e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@288a22c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4c4ca0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@400b4705 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21aabac3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@584f1014 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66d86ef1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c713e40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f9ffaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28968114 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d9cdd11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141457,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@786a61dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fa9edf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@232b4539 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48b47018 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63fb9c53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13cd73f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a36ee3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e967d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4f6ab9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46e5cc73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b7f9594 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5162bed7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143d1ce0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c53dc40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b67356b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f37b4d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@239179cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c21022 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63aca91c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1df30bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29286655 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78ba2052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e589210 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141458,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a3f88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c0afbdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@459beebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f692c0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b8ca14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d710c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@923ea17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9cbe61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@511a2bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ae0503b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c857e22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b0aa65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f5a214 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eec4d79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9df5ee8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8f3e34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21f1ce73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d5ac8b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e81ffec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d3c0096 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1211d314 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d59ea7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@385a76ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141459,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65bfe9c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@792a1f13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cef9637 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77696e42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d02709d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44f928a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2447502f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18e0219b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66183c82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b29ec88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75711fea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c840426 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bdfd918 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f362147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1245ed04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eac6e5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b8e7b8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356c43a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8fca5c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359ff41e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d558b0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a738468 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17cce44a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141460,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d39cda9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3520e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a03bfc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a1de27d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59da2a12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c1d7ba8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc7344f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eec2140 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26ee0435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6169fb7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53bede55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c970343 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a0e6348 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6492fb51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41fbcde0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65d25387 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cff7856 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f4458eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304f042e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477637cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606cdabb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de2455d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@254442bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141461,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53fae1b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55c55071 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af22f39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da4e146 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29d74a1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566ec18f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5ea79c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67fac5a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@746ef4e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@542c3152 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c560e0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7db005e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7efa6f00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4be9672c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37b33633 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a51a23b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@677e561e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672e8dda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49023da2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e051ec3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b5e67bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@88d7957 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50baaac9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ef551b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141462,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6497c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dc01298 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a9030b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7462c688 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e828541 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@332b1adb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aaf6ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@611e2695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b6c1f7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50fa270e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b3b679 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@148912f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f4548d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e06d461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@579fb3a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bdd1513 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ebbdf70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e6ab1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c4e38e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd8c7f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f28c320 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c350ba4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f1bbe1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141463,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@180f2b4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3117caae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@551b9f03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@593918f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388d1dd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37c0e2d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e2a80ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fc5f114 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ff61ab3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1099c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25556751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25187fbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6af4da2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21e1802b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f0004a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49037b78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56aedce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a7d590 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72db961 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55a7ca47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f149a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11ec079b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be04276 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10fe4abf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141464,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4fad49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c6eb1a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e4bdad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772da61c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e86855 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e12089d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e2bc527 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@242c89b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7052f7a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1a4fdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ef8bdc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5984e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2951d53c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1589137e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ed48a84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c283b72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf7edfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26b8031d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc23de4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ac6ea5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@570f373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b6a82b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141465,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed684a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cad0e9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e448f10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58e58903 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@420cd51f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26b1e225 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca7d5ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@787ee8ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cfe6ed7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75f70fa7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47cdac40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@487b35eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675c62e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a9bc39e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9ed521 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11db6cb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15947f20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6167d93f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a62fef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d525f63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b094696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3478c4e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12875cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5de1a0aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141466,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cfd6076 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed0bc8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26197024 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ad70bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c440ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a7ba4a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78e719d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a3bd149 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ba20f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ed3e1f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a25772b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e504cbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56655a25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2748d555 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@166562b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8cf17f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c43e79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5747b789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a8a7f7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a07c5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5000b527 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a11ab51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3597acf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141467,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475fa464 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12238825 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25dff213 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d987b65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@575e9e36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b25658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3276817 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4a7845 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e46fcf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450d7bf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d52d494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a4132b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3abcfef7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14c0c0b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f5d50c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc3394a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1553f63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1302e2f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc6301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@438003fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b0682cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f17b875 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aa60afd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141468,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@497a4f89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1635ac93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44336a30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ef861a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65657ff8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e8f16f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b285f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c7a494e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72ae7b86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5430d3ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b80195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe600d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e153ce4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6a272b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5f359a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62a0624c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@655590e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@482ba01e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cdcaf64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d8f06b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64144170 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71aa36bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58240062 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141469,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dcb2f59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b12e051 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bad2059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13720a5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47175e8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bdb3e55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f67a9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a55b305 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67cb7def -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2520ced5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f287a6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38523d17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44a1bf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27bf2bcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b88de4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@429becaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@165ac2a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e572ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65b2af03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7851d98e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@385e0bae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bcf7452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b75029f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141470,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d9e33d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15d540b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3598cbbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@367a566a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad6c3f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b9d0da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568f2afa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3222ef47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e4763b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75134fa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae47300 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f99e7cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a66416c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35677bdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce27202 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c754bd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14701816 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac720d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@456a23ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1317e375 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b3f9a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24d1a819 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30b09a3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141471,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c10cb16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f6078c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4ee66b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@191bf896 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9d8d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3737754a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d90b079 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b093a15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664b38b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a5a9c12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@221fa58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@601ab19b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2bbbb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@612f064f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cd1d308 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59723160 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b86c7e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4202bfa0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20296d7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6898a12f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ccda12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52648ab4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@394d6736 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141472,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147138e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778e6c3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1496950 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53efca98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ec4167c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aaf640f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e697e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2df0366d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43e9b259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72d72937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@463ae101 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad5f4f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64a0e88c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f75fbbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed635b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be11ccb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18e67fb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@184ecf1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17ba525e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64ce1326 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f7ee3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a6d301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@644b5dc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141473,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70083545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4851f114 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f780dc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6163a6f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50701583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17ee87c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8e7aedc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74cff529 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24433a3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477039b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7254eec5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b92b1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44967cd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed2802 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@579acc5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67cbd447 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10cc04da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0a83f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a7348e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43a3c20d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33b9e477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c03bcb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cd793d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b985ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141474,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ddf57f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb896d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c31c790 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71caaf04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@560d0e01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e1b3c30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c846f24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7812593f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d12d3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513c736c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ebedb20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@448fb469 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b78f93f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ed4c2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61d2f7e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c72b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14ba9060 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47653aec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31fef056 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50027034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@107bd3c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e27397d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@389d1aaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141475,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b5d4e58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35765704 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@725d7c06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22c55880 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337bed0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b25212 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d96a5ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@251fe9a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49e06394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f649a78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c4fa8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4c6f1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@677a45e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c86a429 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cc633b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@323cb789 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dc914f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e62da71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5374119a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a36482 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6716dbb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6faf2471 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca400df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141476,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33cf0bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52a66031 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44bc4ab8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78220808 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c7fb6a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7db9faff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b76967a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33fcb754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52d7957c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@600bf3db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f216ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59313c41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a33e6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@889966a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ccb8dc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc86b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72366e94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36489c7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66bdbaf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b3240a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3830b766 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ae97772 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@410836fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141477,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21fbc9a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ccdd17f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d9e6773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da11df8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6268792c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@105c0a37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e1742b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bccde57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4acc7b1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab1cdc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e770ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73c44d1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5af9f4dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e92d5b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28f4fa92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7909ee3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc3bad4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1484d5a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bab9dc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d65e086 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c01fae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1a1d11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@291898c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141478,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfb3cb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78fe35f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46324d68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b812bc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@596d8ca7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50357bd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43825b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2645999a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13783141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68681321 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db89606 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27bb4362 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@751b8ced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc26874 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34a3cd6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49fe38c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@602bb9ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee4fc46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4069c5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c06d462 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1df765c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8d0bced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38041de8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32ab7b4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141479,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@483c86d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45867cef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d75c2ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346db301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@184bb153 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e5cd7b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dcd2a02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44617701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e587f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@98c2925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de58f2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f336ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f84d578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772902ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26646f8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d8b69ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@341d4232 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b9e1a7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e832cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5891e09e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@643356c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d758a39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a0497ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141480,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d461f04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2231b595 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7344d7a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db0a4cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30832240 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb1b4d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a1dd911 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72338a53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b0b2777 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@498e4fba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dffd2cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f0ac35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c203a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8a6fbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29709bae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f670bb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea9dae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b7a511 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ab7e9a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d1a4fc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d94ce72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77e16ffc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141481,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@325e625b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d857fa9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3659d735 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4abbd08d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b363f3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2de871c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5570fb34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415b66af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b2e5658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a65697 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475db64c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d29de16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54bccf7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73092993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59cc1e68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a86a229 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78735b23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@420cb00f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0ac978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@462abda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66aa4236 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41810a24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c6d5a65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d401049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141482,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49cf5625 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d6a512 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e791f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bd1f3c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489cece2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d573ed6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@80406e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3fa3ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ebab53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@170e8e76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f15a963 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb57cb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@300ae90e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5314bc47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41d849f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@549dadc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33387ccc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45b81868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ac66d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe67029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f48ad05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a369fdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f89baed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141483,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6322f00d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c859e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@173efb73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@758265ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@154518e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab8475c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a3ddc1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf45fca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f8fbbfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672597f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c420a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dd847bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fba7d9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346bf786 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55066f13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6c37a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7696efff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d2ae16c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb27b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4058d284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f02467 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b1b84b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2398e5e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d2dfec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141484,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2581042d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a81a0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56fb061c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e58e90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48685e6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a624d1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34a777a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dfffea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ee5c47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f4cfd10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50d9d36f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52512531 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57d84758 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f4484ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711870e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2abbbb5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f4cdbd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74e54a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4648ab18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31de81df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68156642 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@367ea43c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35edc980 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141485,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70f8669b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@331c29b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@700034ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfa52e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@180525b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a86dea1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b520e36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c636fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73dff3f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f5058f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2458d497 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@575a6736 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b96061f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@560c6a68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3492f052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30f54221 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bb81cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ed272eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d611f03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@571fb8c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5a1015 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3162665f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74beee77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141486,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e4db1bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@428bc8d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60791e48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b78966 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c414fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c95cb35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b617b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b954ef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b06d68b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292860bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d77f1b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c04926e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@114bb3eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0c14f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3934a061 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@716bc2cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3df3f3e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2643f303 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a686802 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@153a881e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f4259c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e16038a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77474232 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141487,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36fc5ebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2032453a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0618e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@539d9fc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74032b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35e9bcf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b065b9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@245a9c89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2289cf24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a7ff1a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@730ca3f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c2d7c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fded87a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a89c6c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c0cc76d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49e52fba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@506123f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@545a40ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e57c2a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e2fb82a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57084498 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@515333b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14191d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72dc973a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141488,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67e5317d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fe368af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2221a392 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34d295ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7af22495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ee09d74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61728a09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ad3441 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a90766d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60514576 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1586b9b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d21887a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14aba2ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ebb2fbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ed06ada -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f035b22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26f432a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1951737 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cce2b65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29e1a195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e87df16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@702595e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a3adfd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141489,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ef33e03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e9c2fa4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc14d80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f9873f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606dace5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f735e43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e5fbd60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252411eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28a39b2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da7493f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e704556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bdeae3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f103684 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68e976e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@247fc456 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a6e119d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b4b042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4749c40a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dcee5c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77c6ed1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b17506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fd18e4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55db8498 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141490,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15353d32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289173d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b812b75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127783d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e97e008 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d3b31c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f736997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@313458eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20b27965 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d3d9b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70e83968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6308c49e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5042e0cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ab68d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ba4c049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eadc997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69ae2031 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@677e1ef7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a752cd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a54b571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1263a3a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd8ebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@660d793d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141491,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49dda7b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46ac739a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fcd9586 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2284c5d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6401614 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23d595e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f755bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d9ae2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62aff415 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5a3f18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37dacdba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e5bada3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@349fd21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fd7a0af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e810df2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d749534 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5198c34e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53c8c30a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f0aa1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c9442a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b8164cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ec42225 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141492,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ee4dfab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@207b038e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fca449 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb67b79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7f6a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f28f2a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a13cfe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36520260 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e68f8e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ede630 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3285138a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e246b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d843b03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4818ae5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e02c0bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18539242 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6473fe68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c2d96a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6422bcb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e7f3fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b1cc136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b84c46a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a327cde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141493,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35151a77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48fd7892 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75675273 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c4a148d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@111171d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@320bb289 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d309a71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21820726 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b07653e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5477072d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d6decc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce7657c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cc9dd4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ebff90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a216a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1549590c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4738fd6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68009d63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3599dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15eb28af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15b85cdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793f208d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f22bd0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141494,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53c04014 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9a80c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f2db50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e32d404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa3f536 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1663bf06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9cec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cd3b8aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c99f443 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5709eb59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dec88bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c7b7540 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fd3b160 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1798c0e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e481bdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29deb220 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b145eca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a5a1688 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e276e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@187183ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18a7c20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bded4cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cdcd006 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141495,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78a660a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a1f78ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67aab9cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d7c09df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2694f794 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@771d23b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@454b8044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48745bc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@495d3d0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57439724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e936428 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2453847a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c678b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff735b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a90ba4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22589d36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@156c6a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6340d7ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c9d6306 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a4af08e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c81d4f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@99a900a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e88476c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141496,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530b1ab3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31cf6a6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c08b69e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30673f54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b326569 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71e4ec60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58f10884 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51650151 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79543f39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f055b9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f60d3d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6763ae7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e0360e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5f011b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22d8bfe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ce1856 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@500f5650 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73fbd293 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143b14fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d2b282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b45c73e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7607f98f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a57b778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141497,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e755412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfc14d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62fe4ba4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6595dcf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11b4d190 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa535cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a96f654 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1afe18e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3620987a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4558d83c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3801a295 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@465c978a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@748d2473 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77979be6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b78a8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6075a590 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d46bb51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36c547b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48d87035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54c159b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d18ba50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51fcbca0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aae2772 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141498,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2b1171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37e9f6fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c4451f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28fa77a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbd4b55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371b0a5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16565d1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dea6f14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4828661f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278efc9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@314d3f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db89e4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36ab0f7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66115f19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43bfa355 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8c1f90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54b21764 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38b7737c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6acda34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c8625ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18cb002d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b64acdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf9a8f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141499,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16bd4b3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56b7ebaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@688b87d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e03445e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@221b1800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b98657f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f99eeed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b7c64f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415da083 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4966964a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79b8f70d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@213b670a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d4e9216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70e79133 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5910bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34202d4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d07d7fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a5a961 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ff8d10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3280a2d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41cb0375 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d09448c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16e5d330 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21c46d17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141500,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@380d02f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f071c58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@274320db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@256d4df1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75a7d927 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52e5fe17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c1e4968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@235393df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68f60873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@159e563c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd508ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3427af20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c4a8c47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b820844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b3e3a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21e4073b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19748ff4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af64b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@353639b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@346a3df0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9dc0fa6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@702f50ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74500ce1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141501,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11302a26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7878b699 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72daa9c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@699847f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1668ef90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b995fba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c2a6fa9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46daa82f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@179fdb0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@761a32cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29125c80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a1a025d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53b4c196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@741f0d77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd2b928 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25304265 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50fd8f24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c8544f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c4adb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2140e46f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d78dbcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bacf616 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48263fb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ee3c941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141502,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359ec9ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@398e9509 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@402cc601 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3075a684 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7367f8f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e0c0d41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1836d84f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a1c4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49868c2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17bee65b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39e4b043 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c396b70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11a3652c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71cfc57d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d3505af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a709192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@407007f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f9bc6ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ac6aa8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44745641 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbbe95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15c49e3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141503,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a69f4f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c4969a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@151f69af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241b2855 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b0be6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4633e3bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@280fa8c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@552b7413 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@112ceb0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48dbdc8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc71cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13426887 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65408d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736912f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ed3148 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566b6d42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37a1d96b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce4fbbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bcfc19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e309651 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7637a7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cf15836 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78f92585 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c1a6d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141504,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e6d8c62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@787b5f71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499f88f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627781e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@341bea24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1639bb01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23544908 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e8e22f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752fa1cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e95eddc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f08ed0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40fef398 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@485723b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49adcc6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a520c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1670655e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@674ba6ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4022d895 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48094b78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc68db2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67823e72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31b2b467 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33e26ba2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141505,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa0db71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c271ff9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8928796 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@280d525d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74aa6f7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60666ec5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cd4fb68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e89d115 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@197f8825 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a3ccd89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11e8601d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61191a4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44298ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@283e5d6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a8428fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5366869a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4373d1d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19fc21f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e429f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25513562 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7871bd6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ab873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6332aa37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f646916 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141506,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a8b2f42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fe7b140 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f2dd99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75b23b68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1620eba3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@449f2596 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b8e2689 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f833deb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e6fb64b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3618785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39e1530e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fbb5dbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca56fee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43f78450 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70bdc603 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@253e3315 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4203f243 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3da93b57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@233a7f3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774856d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a042396 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e83c602 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e30896a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141507,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bfb949c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10681ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509b6b3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a4ad39f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dd2f73e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d8d434e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66441f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4abeee2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60ff8b41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@676f2323 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2197161a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ce54b91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dc7ecf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb433cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ad1c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3105bb75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52cc919c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d59f25a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c0c9ad6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20ce4031 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@163865c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34518e7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141508,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c53680c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4e627f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b8d2e3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53975f6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24734e1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69e39d76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b1db2aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@298d4a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ca5614d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49d85382 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd620f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bafe852 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c974f63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17d85b30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f73092 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6ea886 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ba4101f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ef0cd80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e53136c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5611b6b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d2e608d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255a2865 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b497d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@584b4569 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141509,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66df6ce5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1232695b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359f7ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37fd0504 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a58ef5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ba133fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6586b9ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0c452b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57435f0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@755f80f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2544ac8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d399011 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30cf039e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@391e6548 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393e8630 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@836301e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40ad1175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ad3080d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47910061 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e9e7e4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5690fe52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@449117ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bcc8516 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141510,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6adee34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@579ef2dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b7cb9bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74776bf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d42e704 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d6af66b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43251923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c42c25c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6326e17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c810914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@786f9a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4c8b28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@81b0c40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c367dfe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4df217 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a1277d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75884685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a8e0c47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19609e4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3556c8d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d0c94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58ebd4ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45648b3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141511,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4677f325 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ca46ed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aec815f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bb0c255 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a7eedd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dfaccac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73d24435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3d8270 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33e1526c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58156710 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@390773cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bfb4899 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b005b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@216ec519 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@237563d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127b58a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40341291 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a55183 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f46eb60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b0166b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f02696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a6450a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2251a765 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141512,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7585ee0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4377a2ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bbe1b36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@447be3ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e54a090 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6048a392 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b347306 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60c9b170 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ba2cdd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e53a10c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a75183 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bc83616 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4649eeb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25716cb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c61ad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7374e37b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac1b7d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54fb78eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5de2e025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d9bfd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ed1f605 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7162e514 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bcbd0d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141513,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f289702 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eb8273c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@681439ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dbc0115 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36170727 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79d9571e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c97876 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d1063d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53307969 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6062b5b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b137b04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45c38d4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e85fae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@605297a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60190bf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39694345 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e2b7ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a300b1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@325efd0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@415ef36e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74b7019a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ec5a4c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141514,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b36aab2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc5f939 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513280ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1212fdac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c6fa666 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed19ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ba06923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eda1466 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ff7d14d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc354d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d2cb38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b058ec0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ac94754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d8bfd1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e536299 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69678a1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79048382 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bc5ca9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6eb73cd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b55747e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6610f805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13744f71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4472884c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141515,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5414e238 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3525e86f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b367d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@547bfe5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278dc3cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d7131c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@608450cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b06eba8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22dcdebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16170c1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d72c21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@850d6b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c3dd030 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62644a5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f3fc415 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4365c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@234ada06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63bc4299 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12522d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32949f57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477bf9e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d2c315 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@456f70cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141516,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53173110 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bbb3e2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b219623 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25f66492 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f834e8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d5b7339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4adc663f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c984902 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9ba833a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ffefb8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26ba99d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3874145d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@650ae33a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d505e97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd628fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77980fa6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4539315c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a827b8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4cd21e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@331febe7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f209f09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7637ed4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141517,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b183874 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5714b823 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf7ad1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25636040 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe22700 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d455d7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c8b969d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d49c2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c606d51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2be96707 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bd62399 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d25ef33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62201d23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2918f3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c1be1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5f9643 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3874412e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3692d968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b6a8e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57dcf58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f9a83e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f6350ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29041655 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141518,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e80b22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9264554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ce7a09a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25d8aded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@276b136f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c5cee33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dae6e52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d790dcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12768b64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6920d26f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1702fde7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32308dbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14bb1d34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b670666 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0a2128 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d610691 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fff0b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e892db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6841f8cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dd13ed8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53551186 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@552d3f7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@324d2297 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141519,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16cefdf8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d6c1786 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51060b81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a04c2ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc4df29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@669935f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a260a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b9fd35b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@666f7c96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49f6e31e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d4525e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8952011 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15837659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b9389e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f8d9312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e39bd5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3006257a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47ad4637 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@344b48d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47ebf06d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4647201f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b765a1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5158a73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141520,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d657a24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62dde64f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ba2f6a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@124da5b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24cf5982 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292d3cca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@516c8b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a89b5a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2454f039 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@645f4315 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@573f1c27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@602c9fce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63fa7eca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d23de4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52523bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56840c87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@663824b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a6fb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4644b9af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30d275d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e8c2c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c90022 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22a35075 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141521,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fcfd20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50d3c914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59920b3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@618ad883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7793c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49acdefd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aee71f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37d76949 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7903b198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56f7d373 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@211640ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d07e8e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49ec13ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21e83dab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32e80213 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20ba9479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e282a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6987e351 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b0d8ce8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fe0caac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720aba69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24c15212 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53dce0ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141522,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a80e5d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dfec918 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33014977 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ec0629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a7fd57a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a26d24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68d1dc96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f6b42eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b7b9279 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9f20d7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f2145bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a082eae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2df79bcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3477f2f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@511e6550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a184c83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ba09bf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30cb5e49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@125d89e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac9b379 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4509e0f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f5ad70a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141523,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513671a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66534281 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cd710ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@154c78e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779598bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bae9b84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2a22e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29d78ad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@725cbf24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d517850 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e2560e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1676d5b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@144fdc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7519646 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1692f284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@366966a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10e5076e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b0d4092 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a8a97a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67abfe7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f4d4b96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e35893d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60cc14e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141524,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582c6eac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d889fc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff68f29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f9df52a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@547faf7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d207c8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26742bc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c0be84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fedd950 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d70826 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@767a2fda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ed9b0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@342befbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d745720 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a57a3ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635b7e0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8e613ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57f49881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3059232f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@84a883f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@308bb39f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4348059f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141525,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a711682 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66266a83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bdc3a93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8bbb28a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3acc6019 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6feb67b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589c07d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50f30da4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@966630b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d845b79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7645fe9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1343dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@457112f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b17cd4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ff71af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66d6cf86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a34641a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f6cb7a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4627d3ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16dd9e50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb02c25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75fb67f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dbb6a1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141526,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ee4ae2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1951813e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@373ba111 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6186fe5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64ba11d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71f7240f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b2e2009 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d0c5135 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@790cc5ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31934198 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd701cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@747d767d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b41ce8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4075ddfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@234f4560 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c1e11ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@594e9542 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37ba29aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e871e78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225ad303 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f13a68f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698229be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f208e28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141527,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b969004 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59237027 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21946d64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a7ef6b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6517434b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27f7b586 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258a1351 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@482caa26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@177eb573 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d3f699f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@785ae3f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550a0b45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@373df9ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b80d53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e5233e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b075d49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33af99ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32fcc809 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4e18a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c72e95d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53dfd120 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7e7d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0124d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31320ac8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141528,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cc99b1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24786623 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17ecc1fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36a9fd83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e8992a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@146b2aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7054705f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9e8ce33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b6b27ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f8c03c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c094eee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3df980a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@717993d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e144c77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8ec1d65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@778ccbba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ad62993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668db34a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee29dcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f2246b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64c37e70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5f9641 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c952a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141529,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7853ad5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79c6cd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e20ff5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0123bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4edd4ac7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a431d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eda227b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c64e0c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284b08fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad1b080 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e7805b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e2e3ee3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e52db3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299039c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ac91f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f4a6020 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6769e1f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1f92ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b28b9ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477f89cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb19ebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c0ae954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141530,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4273ccf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@797934a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ade114b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11951b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ddd5191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d318088 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@178f922c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e7bbea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@98d3324 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7849d75d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13bc348a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60c10ef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3505d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f0d9b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@449ab383 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c44d326 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47760c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347a0cb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@256db71f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f3a9d77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1074a914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@740f786e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c12ecbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141531,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47380f6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72b13295 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d72e8a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@562e94a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51eaba7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ab40f4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40e9fd7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62c0ea55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66fd9d22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cb938c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b42a33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c3f652 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cdd7f25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2907016b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b2053ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72c8599a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@158b24b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4967332 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc1ce69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5df55ec2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11174761 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141532,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26afb992 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45fe89a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@328adc9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@198da9e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42b5f06e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73379259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17bde6a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@638baf3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed005a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@413aeb7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@394a9d1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7061771 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9f5d719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f82ac50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@724d68c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4547546f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622d0ec1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6847cef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71d5f010 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e4e3669 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41f6a67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f95412 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c726b2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141533,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73f0e9b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c43f9ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@140275a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb64f0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fffba84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce02885 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bc8f7c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52bf1384 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5096a8cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d5f065a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1185a8eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b0caa6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e39bd00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b911d4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27ef2c22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79eead79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a2cdef9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79c4fdb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27944094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b9db47a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27ca7ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79bd2cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c284383 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141534,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad00618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e0960e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ec28ec5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6048fac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a12d012 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f99e598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e7fc08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@325ea2a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e8066ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ea5e9d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44085fb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@304b62f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e4ca543 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fe6100c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@399505a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f866973 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbb17d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b92a79e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a3b1b49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36da248c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f730b73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7143ef54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33605366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5479ffc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141535,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23264ead -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d713cd2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3bad79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6727dad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7665a3bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3236960e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1347b554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4aef14ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11cc360f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@174c44de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@390fae76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf79f86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7bbcb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee46dbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4adc1e08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de0a37a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bd8560b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76fc5941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cfa35c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24f9a65e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d08c4c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@292ab475 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@247380ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141536,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d51096b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3684cae1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4af14958 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72bddc03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3762996e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27219ea1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675aa894 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9b2db8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5891624c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@736e3b69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41423398 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631630e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c6bdc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f17dcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62ff46fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30bcc35c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35910877 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb7242b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e74bf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4217cc16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ff8a693 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a5b433b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64cd1166 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141537,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18461695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2230f730 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32e22ad6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea85bcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f20033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33defcaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c9c49d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c43837a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a36f559 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e496f19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70faab61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52ae57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d131f39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38478225 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4387ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@876a206 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d598ee6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490b7bc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48f6f1f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ad1c79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74029294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66c522bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19c33eff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@410a76a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141538,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64e6134e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b7915a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b4df2f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a673b07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ecf3166 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d78d727 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59666411 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a4ac339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d098f17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1613a87e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cbf1041 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f047e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61800dda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6adb09cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e6221ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c4f9481 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dc0a60f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe75712 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20747171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ef0262 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@387726ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dcdb975 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eb1a444 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141539,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@766541e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77a65bd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47105246 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14504d64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4866645d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ddbee6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489c0ff3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eb140e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@250e93ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7774b1ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fbfdb27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b5d7e31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a9d1d57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8362670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2971b2b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49bdfd5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf8369e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c48a37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@660454ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40753a41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a03a817 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c701052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58171c8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141540,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31d992fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@567dd7c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a13afdf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a802176 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b318ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc80649 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38f23138 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d22cc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ceb80c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733e2f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aecbbfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@585a2045 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58370e4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a71e6ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@657d9d4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cacd1af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c89ef03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@599824b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50064d3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a2dcf3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@199fc28d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bad0a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63b73233 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141541,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57d75556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30bb6e8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67d39ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3daf1f94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50bf8092 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b9b7338 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@404c6f9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55a8bb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@294d2090 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@492f8432 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a3c1a3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bea4386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b63adcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ed8a881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@443f8ea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da003d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b4bfe5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405cdb13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78ae0f2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@553d2562 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4110a0b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67326fca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f6dc366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141542,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5347eddc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1796926c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63aef36b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62d96a8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9c5201 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dbab44c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@391b3f3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10bf87a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cdfd3ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bf10ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64bd15a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8d3041c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a11fe50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d5b79c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac994de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aaa8c7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f4c0c04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34fe2f26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a72203 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cd09f5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337388ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@763116b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dbd8546 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141543,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75afc9d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a341b71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24ce4a50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5786c3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e12d41e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7407b93e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22cde124 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@641240fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb8ca43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@353f52a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b641e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed5d394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3405fadf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7be00cb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3113e362 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@117acfa7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a320c55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@714c3419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f6ad1a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@182a3a1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0dde8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@129ccd51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bf2e17a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52c1c2c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141544,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ca4aef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2248e322 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3d3f20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72363ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@762f8e6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@139b2278 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5498cb14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17128ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d443f97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@578656a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d4f72a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@463ab550 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7abe8287 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fe8c00b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de7097b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ee4a3a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60c0ae43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3971d2c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c98a84c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37aaedce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527dffce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@652d90db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4e8efa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141545,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6acc20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d4c7ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8762445 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e742ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a86911f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dcac404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@542791f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4d41a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30bd2f1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b6c37ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@632cbf45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47489aec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ed4a8e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74ed2788 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@723d78c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a10396 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8926ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c59954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e27382b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37632fca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34dee1ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f2ad14c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141546,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@383b4ee2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16082b50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5518cb1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c21c044 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8fdb7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@798a3756 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ad97366 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b8ad99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@783cd7c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a46f1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71c7742e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c7edbea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@262d533 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@714d9ef7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be1e2f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512acfe9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45f543b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58d2d7dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c107871 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b42b79d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4459194 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a67c2fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141547,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25dbe833 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bdb4d4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733aae94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73ef23f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d5e089b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ceeab67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a86e460 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a594a05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27ce8505 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e72b708 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43bae342 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23539052 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7066df90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b97fc63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f23b6a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@604d7c16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2611b5d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52113195 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10450bc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1be36813 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3077e18a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb1e388 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5392f63c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141548,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5331afad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cc56ff0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388c7a43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c6b4878 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22f9dc9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b48ddf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eb1ab48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@283f1a6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ce26d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356591e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368d87ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66b1173b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53fc5589 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8f3c8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10e770c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e01a52e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f60e77a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c38c954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57f2e755 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24a90050 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cba7888 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56727e22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16faf3f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141549,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7abddaf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45fc831e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ba76ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@411e504f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55603087 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74a5bd46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@277bb021 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14d400a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40e4eebc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1972346b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637275dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5717f053 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30616503 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3838ce08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48948046 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e9746f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b1b540c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65ad91e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d7abaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6276ecf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@230c7f85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aa780d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e99e4d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141550,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bdd8285 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b6d2c2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d5aacb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c346ca3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b5a060a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9345b01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41437196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2efd4c69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ac75ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e6bc80d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@647ec197 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20244775 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@565165be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@361ae4d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@247ae9f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26671de1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc6f006 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a1bff8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69dd8075 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774b00ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bc19895 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee3dfc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b383640 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141551,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54d9b74d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63eb8873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3164e18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39a6e20c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@205c49aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fcbf5c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e9c1ead -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20a42ff0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62773357 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2252bf3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf5a4af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dafcd6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3699a607 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18554a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@401458e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53c32937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@281114d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31a4d0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@612bf6e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bebf163 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bfcf76c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5fa13b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@647dd066 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141552,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10e95563 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c429785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b33b74f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5147a2af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bdfb7a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6349f7e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2506bd01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49b821d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2cb5b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f6e30f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388aac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb69de1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ed103f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75276f96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33bf3d75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd9f776 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@358c723a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b99f0c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31073e51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72a60d40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24edbc8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10464733 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141553,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183b16dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ab5d496 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f50a90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3128a802 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7165c50d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@580bed3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e9a8579 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a065210 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1514abf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15e2e9a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fb9172f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d74c1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37ac2c3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@350154b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2363d58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f3c1698 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69a738b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@563c15a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65532eca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a847c2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a36a176 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9178cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@204838eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141554,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393455c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66567e81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31563596 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50b8dc37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bb919b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eced435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb4a8ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@87b867a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b0c4271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630b5059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c3ff97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c5feaa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65292e93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9923f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1be140b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6898ab99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44695daf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51fa484e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@408f4625 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d66f9ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63ec45e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60d7d9e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad305b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141555,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a13faf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e3f956 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f253048 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d99508b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14cc48d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a7b2a56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7caaec21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b60e637 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d9ca143 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee6193c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25863725 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9fb2cfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31d51239 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fc4c2c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b6fe304 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ba3a059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527e27ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b850db5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f16b602 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49616cd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a827264 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39392224 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b535a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141556,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f0c4e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28189dec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5840b2c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6262ac6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35e29327 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f99f8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@472a886 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73044d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@525e837b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@474e5375 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5669e582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d0622d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b37cf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14d3564f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d0ff4a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@94a3e24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33a4bf3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dcbf62b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f13bfd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1770130c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b1393 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6323d731 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@311c399e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141557,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e3a4c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f1d8c06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27a468b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527a4f93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698434aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a6af820 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a74b7f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af273f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@629b3362 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f9d6ce9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4059c995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@798d5f8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@423673ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e8b574 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68f3dd5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9196eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a49abed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29a9ecc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fc6c598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28972645 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f46f679 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9efe090 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606c1e30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141558,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7158a976 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29756c1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f68f2c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f75fa87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd2eb06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69c54fda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71236ef4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f20395 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d845276 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e7c6983 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5442e476 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@453dbbc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af546f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4798145f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43a398e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c716172 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c8368d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bbeadf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e9d0142 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69c08312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e82ab1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d5a04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@260ccc50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141559,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4bb90390 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3098a38f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a1f22e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a311bbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19172ee8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ce6729e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fcd45d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d335365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df35645 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22deb9e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7189bbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40775846 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27516f73 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7017fbff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33917305 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49dea859 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65586ce6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a5fabb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12eeeca4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e4e12c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8d774b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f82443b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a6c18a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64eac53c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141560,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57ab47fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ac562ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40b6c857 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72c73658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fde1b49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e3fd61c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38bd4bfc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7383df45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c88962f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c373094 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e876045 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7313fd27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b922484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e541ef2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b74c435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7762f8b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d8e54d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33336c18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@edb4327 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e9cd575 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@316b1c8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1910e516 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d065f68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141561,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45390cc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cdfff8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32798754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d7b636c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4029290f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13598d19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de4fb01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9295d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@727aaafe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d14c0ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67621abd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b13f86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fbb4934 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24a3f786 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e3b65d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@522feea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2728ff30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c0c3394 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52932f16 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe4d142 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fd570f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62b8600c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299818d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141562,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70afa262 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c9167e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65d0b6eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce76c0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d66347 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d20efe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6da76744 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a17c9cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68e94591 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62bcc0ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34f706fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50cae81a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72adea5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7610fd14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5cf5fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd14893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b3aa64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@113a133b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c54d7c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141563,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b36930a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6abe682 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47499ea7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d09a342 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22e8c32a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7baeae03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7099ab77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50aa0595 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@343f6d39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a6f030b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@710c522 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@796eef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@480f1d96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aba28c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b02c567 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b531bd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b005280 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5027af37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9371a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ebd7659 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a0bd5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141564,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a93e888 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cdbde4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59517436 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bf9681d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3b0042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c10b241 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e329331 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@efba56a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b48de7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@511fba0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5e0003 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@685d32f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345a51ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183962c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@532c27d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52676dc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34d68102 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672af217 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550a03c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cea65c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c343dcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cbeda14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141565,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9affc64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668444fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b43d7a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@157cca1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ed5c33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@330a4d9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56914ab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@691ca601 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@748d63d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@150aae6e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49929283 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4532191e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a3e59a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a07f551 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47dc7a18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12737114 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a28f995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fa02efc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b889baa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@327ab61e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fe30595 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141566,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@512a155b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e00811f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@492d0887 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb5739a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f342dda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35164f5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14de004f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21c288ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fa62bf2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18cddbc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cf57ced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4945067e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67925d1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11f791ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b2852a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27251d74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@522ac62a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f7f4d7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db60c6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22777a14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2668f3e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f976c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141567,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb1e385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278f999d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4619b949 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3eb011 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f22c398 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44baf290 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f62c8bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1487577a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1db07871 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e8d3725 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5773317b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79e18068 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@678c1cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f2b1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e693a87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b7e01e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c93d8ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@456d13bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4257a493 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9eddc2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b9b3143 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6460f40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141568,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@794a4027 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e21cfa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc2f406 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ac8dad1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a8f2d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61e43fc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a7d853b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ffcaf7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3077aeb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70ada569 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60fb21a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7edf685e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73506336 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e4b16d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15afe669 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10b7e11a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@608e69e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56455ef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141569,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cb376f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1323ef44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711de8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3f41fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1ce43e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c969d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fdfa09a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59e93a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e80839 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79734a8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b8afe4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b22975a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e2ef7c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d6adf9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19e30ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e59efd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4750bb12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c1d3264 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf4e0d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d42bdf8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f881169 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4af38aee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141570,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14a0100 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@698fc1ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f386ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4001ca67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11929166 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@687b059 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c4b88ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42d739d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@474b894b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cf5ec83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c28360f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57cab3d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5069b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f12154b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7267c0e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7031916a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f634b03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3054a9e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cf84f8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e2bbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a053e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@302ead54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141571,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9428cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f868973 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@589eba78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d1fe832 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32e2791c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79e4a322 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54848b1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3638b7c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17330284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b124ef3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3ee0e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@484be6c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e3fa6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1edf41b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@522aab5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45c02a38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da8906d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@799eb09f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e69ac84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3abcd89b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63ce0c59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fd852f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141572,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce6de04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ad03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13fdd596 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1f62f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54638dcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41dc18f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cf0343f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@793de997 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31780e98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1042b3e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d2e2c1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bc90978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a34dc57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fecbb42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13fe0850 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@608f419e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb5efbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a973c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45f643ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fef695f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141573,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c84a511 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11dee629 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66412fef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c007e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb59e8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d3cdc7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12edba63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50d01898 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6726a5f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c625a5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a3c910 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53671524 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc7bdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f626904 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7214d6b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19b123e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb1d993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@767b5a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53125aab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@432ccd18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36a4e9af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd4d026 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141574,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dd20415 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@683344ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79cb2182 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53b48b7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9bcfb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cff4c4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c98b563 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b0022ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b368e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d2d287e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6407a915 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ebaf029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1302a0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ca8c8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5cc026 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55fe5a3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@559e42fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14713216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e069d4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@198e0f08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea00751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23dc6311 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141575,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@293a76c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4025daa9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e81c773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e05e759 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44f3526e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a6d38cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aa7d8aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b89c388 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@153788b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56df51b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32d9a141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3749a111 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e12e56d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@326c3586 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dc05923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@377b28a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4b9aac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5858e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@229e0d2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c4e07ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb27a30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1029868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53270a12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141576,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fea0ee6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52b48ad1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e08937 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cfc6c43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13deca04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@654a38ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a0bc03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f142e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c1a2b30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3821fd47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc20ee4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c7a93e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25595ebd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12255494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@199bdd1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cd0194e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@339308df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@baa6bcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9e14af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10586e1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ba6593f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@630ab985 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f8df227 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141577,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf6677b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@510a87b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3360efcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34c3f4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c3e4de1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6d796d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@847f7aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a930fb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248c0647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73fda5aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44b0f6d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@362c8778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@725611c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c183ad1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f5d17bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d985268 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f82b42e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1261dfbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55c6daba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a72d978 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40fb33dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cfb8049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f0b9266 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141578,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@434e25e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a329b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a4ba147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393080ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b63f31f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45438768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4285471e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f0ea954 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be146a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3240e472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a95e893 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505bae9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@316e2e78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4383f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711c9e37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3ee1c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c630dd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79647faf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c9cbb92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67a75716 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@430ee380 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19cd6549 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10041dc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d46e738 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141579,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79e01ff3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bf5caab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b2f7449 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@412a2b33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ccbb62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dc99e42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d042c8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d67e7ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10ad52cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c7f512 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ddbd1f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ff30b7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35322345 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ba32f94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28947309 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f6b15ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5be72919 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9ea727e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed5471 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258c4753 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47410d1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1de2fe0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141580,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75447f1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32b5183b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dd3eaaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774b7990 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a15ce1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b4ffc2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@592bc757 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7500a21b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eedbe0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a078771 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d91fa19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6df193ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e5f4db4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78c56501 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a161932 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b98a879 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f8e7653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c30f3ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26b5aaf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68a0153c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34c0da95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f95e140 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fc3873 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c1ddc25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141581,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ebbbf41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ec4d237 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bde423c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19e04196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ae2a45c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ef805d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b89fd9e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30fee588 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ff1778f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e2fd44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65e385eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2276ee7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56cfb8b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@405167f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39646ec8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c7452a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7554582 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c70ec5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5678d933 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5817600e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c357db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1a7f0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3b98bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141582,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d4f9a64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225d6199 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd298c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dde1ff6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c732e93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45ebc72d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e244bc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5348d65c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@465a7d10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d931859 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61553a69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@743afa29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619dab43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@248936f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc7d673 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f0f8270 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43c4ff61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e17371e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3118e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1e1915 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b45c860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cfdb5d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@487f44bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141583,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77963a14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7215ea4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23178ae0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@117f6881 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577fb620 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ae5611b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@153aba2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7febe36b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@558f8400 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c3da13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e8fd74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14b8b039 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d3e892a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57c7cfdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24b79b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75dfdeda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce17e92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5531effd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dc679a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@466447c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7146b479 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bd2f3ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5527bc25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141584,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3119e347 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75b87c61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49cb96d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2165cd5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9811ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69752508 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c1d73d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@731d4e36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c1536da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c880335 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b9c56b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2269c30a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@670f927b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b72a404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52824205 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e1faacc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25986b3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd4a90d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b064618 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@380cda7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eab97da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4029d1c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d67958 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57d78d36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141585,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5540c551 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b69a1f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61cf1f7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a1a904d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5923b612 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd3e2fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56fe289e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8b852f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c6fe6d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cfb29a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b66487d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71839217 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42e98bae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79a2c5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eb6a970 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6348919b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d72c2a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b5ceb8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a28b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2779a568 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce52cda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b9947e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51372b30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141586,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b0f869b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ef9eedc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22f8919 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a294cc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2dd52447 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c5bb02f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@524283fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@722da207 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76de5719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7755b25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ee7554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e4cdaf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a2e38bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77affca8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d973dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dfa8ae9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56ba8e0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e7862ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5031bd4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7935d863 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bfdebae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@660e880d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b08924f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141587,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ca9b38d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@781d952d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a4124c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1524492e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b0a5dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2010fa9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4394b567 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74cee902 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45613c02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6282da1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45b64a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7de0bd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b13b601 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f35d757 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6014fa51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e3cd2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4116c26f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50883a75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17099447 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0aa127 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69781d33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b2d9fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23c2c106 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141588,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@688a5fc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41b24667 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fa122ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a5e0ca8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2010347c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73c2df67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55d00fce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f817290 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e1312c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bcfcf8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb40142 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a95e3f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c96373e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a008b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1790a4de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22af6f89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fad8325 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1651a7a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f547e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d7fb3d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea8a7f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@255f236b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b3bf226 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b7a69e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141589,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50968c13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c8ea911 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58f0d064 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20bd9fe8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@442f79f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d10964d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63bd027b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f33933b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10ecdf9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29779132 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2252f0bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11cb1587 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672d381d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72c6dbcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a40a36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e8a4868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e97474 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4435ff21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23a7b82b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f327d3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34b3ec79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f94c139 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ec4c00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141590,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ddb1a87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@620fb61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f447f95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cebb031 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29919609 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28eb9ba9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@740fe1f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7564ad7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50e04c05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2422a363 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fa9e640 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f76b196 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2da44a19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c2cdd75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cd7a356 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3667575b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@391c270c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43a1822f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad441e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e7c169f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5600741b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@197f340e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@719a1e8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141591,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fbc26c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a492989 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af6bb8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb99619 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75cd73c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76cb0998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@697ed620 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb08c48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@307df16c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e54b932 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@104b30e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bfcadaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d5cf28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f7365a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bcb238a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@259452a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f6275b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b6809d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@190da84d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e8bc8ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7228f927 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141592,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1965107a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dde9cdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d5057c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1884628e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29535724 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30723e91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fb3811a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24981070 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@174189b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36e6bb7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@355e57e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a1daa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ab6c52e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4988a244 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eae4d8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5105fa3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2de6ca85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21f0c7ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d4e1c56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2364c9cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@384d8dc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f257a09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141593,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a46917 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@370bc419 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@637c6cb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@573da7e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@464080c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f02af1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f9b778c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a3cf31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57a71bc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de73c6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19b4114c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@12e59f0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d73d7a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29a3a807 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a4e20e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8702e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d2a2e78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79fd544a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7e7de9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58f8ae85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44bd120c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577141d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a5dffe7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141594,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@774cf8af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bcdb938 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ef247d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c84180 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d158d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25e327ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5023e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b55ad6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2db413f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e3bd2c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65afb9c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a76ee9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bad1f1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6be0514a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7319845d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b217cf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7817fd49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11948900 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e10dc03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e46fbc4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79fcb4f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c95ee46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@644dc313 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141595,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ddc9c85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711beefa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@129fd941 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1840d55a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11714a63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3fb30a89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530a89ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d7e7cd1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5647054f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7107aa4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76c50072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e3628f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26e06ba1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a79e0fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e1bb56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c5505f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42dd9265 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6706dcfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e04f040 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@138e0477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d6ef2d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c5e30bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15214b86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141596,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76ef48e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cdd94d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@219b9029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e7982e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@325d2dc1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62f66c71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f91a2de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d0d7571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36657e34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6721196d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7534c005 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7314852f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e1dc1a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1984778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52ff8ba6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@515512b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505b8d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46e59c88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68698d49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@475a1566 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb1ce5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5da0ef8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141597,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21fdd8cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@361b18b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b9d334 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@241aaf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d3521d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23856647 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3abc2170 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@352522d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@517aae17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59454d21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a940d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e86c1ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a08ce2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69d7a40b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21b09669 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9912299 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b4f43b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@117e57a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24b128ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2434430a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56537e9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68517d1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@450ad6ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141598,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43a0f113 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ecaf85a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d90dd10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ebce187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ad0bf2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63018802 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@892a12f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@113bd963 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e2d0d81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4103c1de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c859b08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@133ab13a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55bde7d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a6fd181 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac2461f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c379b1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60e306c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72191af5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a835817 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aeccf20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d960141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57e270b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57a1c9bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141599,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bc889a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76891995 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@322ab020 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b1f6beb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f55a85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a0ebe3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc2a2ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d92da8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@639d80cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c2d35c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b03778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f2dd7ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3039833c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ff3bc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1011c9f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4153cfb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d8f84cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3122b1b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f62981a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7093bc41 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31fbe772 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15703f84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee9d778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141600,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43f51750 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ddba82b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39c32e0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f16c5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3255bfa5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42bfc334 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622c1e1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@306d1d8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fda24b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1340e418 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@111ed480 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49c7026d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37c66065 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23b222a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@446f4883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@354afe09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@318b658d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39efe37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d8b8597 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd38e99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359e3fe3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41051021 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c392ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141601,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6327676c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27823dcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eefa505 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720520b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ac13592 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c9b115b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fef4840 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74b4a5ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@577a634d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132fa130 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d7565d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@179d10ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13f9791b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d317c75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@128b830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ecfd0db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26264fb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50825ddd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14af96c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14e6d583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@290328 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4287ca45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141602,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af6656e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@89ee77e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40669506 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ccfbaad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@258b654f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad1d397 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eeba36e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f306ecc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11efcf78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45f6fbb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3675c0ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5f9eb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35241e8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@477d00a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f1568e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7999e79b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c5c766c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@438b2363 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@372d0993 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13e95d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e5e7c8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1454d61c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b6a3494 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141603,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e8fb82e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43c81174 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711ecb27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b4a9f04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c7048b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b87f356 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c4742f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134a5ac1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b9e77f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17b0b91d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17639cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40abe776 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ce56203 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0f4ad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8780b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31885249 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@165adb01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e752cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f78e6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@139e8222 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53dcd183 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b65b039 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@473dc1cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141604,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2055c49b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@153059db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23e2c73c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@af8512d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4073df5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@136e522f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2680ac59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e39e09d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@536c803a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f25f52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1133df31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@302fe882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@775d3ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27dd149e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28989d03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67bc0b6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eff56cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c9d26e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5537d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a35864a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e48dcc9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11dd3fcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f26999 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b3a2bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141605,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b858a90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@735ef773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35a6f620 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b0e3c55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cf8f0ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1819692d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a8c9bf8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50aae068 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b8c0753 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@588f470e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26a254bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@218c49a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ad1645d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68c4fd3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c77432e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ff5d049 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e47bc66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1125879d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772dc8a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@466b3990 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252c09b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18887be0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141606,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@179e8bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e41f6bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b653575 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e90282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e1b3e22 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f7efd74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d42a573 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65f75f4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@720a87f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a375191 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e60e99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db41889 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b277589 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7c14a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@293c5f95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34517a6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14faca55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f382d53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44331bd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0ea319 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3964664f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77f89685 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b49de07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141607,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2551a61c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f9deae7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c620f2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69be5142 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41777b2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11ed7b34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dea0551 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@164561ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c91ce0b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@165f2c76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b427361 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@763529ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@143650a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b65e2c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23e45fbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c41e0ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad42cb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e73f680 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@feb515f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cde96ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7948449 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1642a5cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43767ff3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141608,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b96e9b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68039a42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bec3642 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b15452b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6732ccbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13979b93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1943e185 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@284ed39b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59e8b50b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@760cf6a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c56e2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec9b9a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3295019c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23bd68ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7852eec4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bb544cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1362fc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4906df57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26cd3c7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@176f9f4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2717d271 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58945da3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1775bc18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141609,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c0bd445 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@659615ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f9535bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25bacf2d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b19e23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52ddda57 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d3155a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17e36d4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca8a913 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@209bc7b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c9e7f3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@501bc0db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9a21bfa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77fae322 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58321d65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a113a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fc5bf45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f06b66c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@271747dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79f56c8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b28db61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4147bbbe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cf32ded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141610,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e30b3fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@539e928e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ff84908 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aa1855f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61854492 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4628b0ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bf5010a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d18f256 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6728eb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dff073 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a14c93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ca18453 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb48462 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6492a3af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@685bae7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68582c3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52f522c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@230b7ce9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16162e8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f86c9ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@449e07a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c0ad7eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bcffca8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141611,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@128d8dd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@269a030f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c452db8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@288ba224 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@561669cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20e29a7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@452f5b33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c05b30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@557d1ced -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61786d72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e3154ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c9a8d80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51bd6e8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@730c9a32 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27cc3cc2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f824336 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2571a772 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32bb64e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74d7745d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79c10202 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51a3707f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3856a2eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@776c4c20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cc81dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141612,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca629f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@416a9b76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5472c237 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36d2730d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b463d95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e62bea0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c43ac21 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@392cfebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@437cc69a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8a0f7eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@983607a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5704aef3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7269ea9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df94496 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6feb7be3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6a23db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20ea0ffb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17803284 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141613,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@230067f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f38fcb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f5da37f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@95fa602 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c05f1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@135cbca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7381c6c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71a110c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c89b67a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cd43ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e07728d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c29e30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4899c714 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21208d2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64f6009d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f8212d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fdfec61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ab34349 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e95466a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36fcfbdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1660b586 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141614,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76a90508 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@728baba1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f89c37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30a7d14b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e980312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@216177c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@469f4efc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@483cf2ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@672e86b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c874809 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10531bcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f5738b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e77be87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38267c40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30a3a5cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36b7a9e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25a7aa31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee806b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c84b6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8bb3a6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10d5ef6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141615,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e66ee02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f344180 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8d4c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@511c6846 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18c39a99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@566e040 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c2ab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33602c88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37736f2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5ed61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@659c36f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57aa42a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b0c456a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5168618f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5547e6e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@507d3991 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@494301e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bb5755f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@312fd80b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@827de0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@593a8ae2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29550317 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141616,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66f579b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50941cc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5df4e1a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cb81cb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11cde658 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@614fcaf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4cde8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78d1cca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@790cd545 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668bb0f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b4ea20b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74ca86e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bf3c3c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50c4ea60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ef44489 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ea91815 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb10dee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c517d12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da21d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56a26bbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141617,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359fc2fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22b180b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fd1c3b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a6b8643 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a82dd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fb29155 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ebf0b59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b0c953 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7876ad38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@335a35a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550685a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b023404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66d04331 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@485103c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@299b94db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@130337f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ae2762 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37216546 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42691235 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@485f5177 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4cea29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141618,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@727bbdf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7889f133 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a6fc3bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@635b757a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c60c8c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b57b811 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f11f6e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7759bad9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db56205 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b64d292 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b82bdcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b75717 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1704b86e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a29085e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64eba00a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f5f4c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6daa5822 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f43d631 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5691c023 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7388c2aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c5cefb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141619,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ddf5a55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65970515 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@503f069f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b354a4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c7fba8f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@611dafb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49be46fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@180812b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@596268f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@550e6c0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e3c79c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36792367 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c16c09d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5ebd0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57a125e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@283c566c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e479de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dfd165a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@132d8404 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59e82f3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18839751 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de90ddd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@421bb3c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141620,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a12dd4f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eafe043 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bdd89ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34c7aaa6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7873f1d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ffab61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e28c909 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17b3c2f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70a96c83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54a97554 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c919e49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bc7969 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ecad029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d60171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2135b0d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4724e2aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f0729ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4b68e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32f4ccea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@522549ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44fbcac6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21929eee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359595aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141621,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d05c29d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57329c01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e20de7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc3fcd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18b1c81b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b208d98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29df3e2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d9e429d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bf9aa25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eab08f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5414c072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b21eac7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d9fdd04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c2064f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ec52656 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a56989b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39221528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160af118 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@610a9219 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f165ce3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b48f7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7773483 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3258f42e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141622,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f7457c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d59fa3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3334143b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46b86ff7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bbfeaa5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@497571ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a733136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f312a58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2baf19d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4937ff4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a0e9ddb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c3b1528 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f89aea7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7932ddb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac670b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bab083 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4184dac5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e345803 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a44e6f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cac7992 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b19ca30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77e17110 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1aff6273 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141623,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6560e857 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509df681 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c886473 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d85776b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f451cdc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10d3ce4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f178bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19cf8578 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e6bd961 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10c4ed71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ceddcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@364f7688 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76af01f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4137c530 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64325067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2984915f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@272abd17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58cbe287 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e8c2091 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@238f4276 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513f8115 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56678e6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a0b49a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141624,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4065c92a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dffcf1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@701203c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b995022 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@755be6e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664f604c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68c1ac79 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3908e903 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@134e7436 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e1e3b13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b59fccd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15675054 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75e8690a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ad11207 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e4a3804 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1002869 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3716a17a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35d5ccb5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a204813 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ae3725e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@487c1709 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d7dad7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141625,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b7f706e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a5f80dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62b312c0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fdf7d23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6fc380b6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@592cacfb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f3900c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5731d734 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24042b54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1873005b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@273f90a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e6e7b0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bb5a11f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc417ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20256ae3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d9edbf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b752a0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20d509cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b2c0d44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70364acb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22413b8b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a52c57d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61dfab5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23fdc5bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141626,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8cb2f2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@340e7a62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2897b2e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@650a526d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e98b90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73f32a05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66e6a61f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51c80994 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2985ad36 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43cc0901 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10499939 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15ca60d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4570d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2693d2bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7acf1701 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e47ba9b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ec9a61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d291ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a7f9c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20970844 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7df5b1a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@683d0a0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141627,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5468df70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ca52891 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c8dc039 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c60dcd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee10376 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b65539b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@199bcac6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3872b54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fb62e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44c3004e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3493fea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39ae695c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@671ce8e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b81553f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@717d8bca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bd3e830 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b084c50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a11dea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17af37f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3399cf5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a0a7fb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eea5496 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2094fa0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d0a3a64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141628,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44e42998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58b68b76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f755ef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53a08666 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@366ddd63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a6c1de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f9b7ab6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78df5a95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20810628 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a3884de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8203e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c8be752 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44130cad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ad37e85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64b0cb11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57b211a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b98099d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a3e793d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68b4c328 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f4ce09d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6453d008 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dddaa2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@533e973e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141629,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b3ecdcc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61979d33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@116bd7f5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36cb2581 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bf6ce39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d5ae71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49ce1521 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26c6d6b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2c4353 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60a1da61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59ffbc49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@491ca2b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6083461f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e1ffabc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a2612c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6831c30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cca0141 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@df1de7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175b7d00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9a79147 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@254b593b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@264a71c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141630,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ab5bb76 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fad32ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20503a81 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55e19b28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1248f4e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24459f2c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5603d17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c01de48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93b869d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10fd469 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@583d9da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5722b4bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@278dfab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dabef11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66fda477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24387b40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60797deb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4288d592 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49dd0fa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ed89fa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56539bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c548c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c4a719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141631,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8b4976 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668f294b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d27de24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@485e44f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@580e6472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47d4db49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3471fb69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e3270e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2534a8a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44db31d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fca3098 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e2d2dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62133fa5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f5b9fc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@513b2e44 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25bb628d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5eae8f88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da8863d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7569dec2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb875e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214069d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18609aa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23c4d8a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b1bee06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141632,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49374414 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@551e7a13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509aa1d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64451e18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a1f160 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3149df65 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345874a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@782109eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6013dd7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fde94ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68542ef7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@184a1c47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664cca1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dde4bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4434ff04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cd46be1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c484444 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73377ea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10987315 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f3c8984 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71189043 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@399272f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ac1be0a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141633,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31ea7a7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4457e89 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e6aa935 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49bc631c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b67d1d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4794305 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fa161be -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55af59d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40d3dbd7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e7cc9a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a24f543 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@556a7a23 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b390e3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63e0ff2f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb390fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@388e5e6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b85cf9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ff85e47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dee113d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77d965e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2275a5b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fa2cdd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32884ce8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ec2e918 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141634,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@115ad6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68307c9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@182f927f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bc224b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2deeb1eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46d1e45f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22893a90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4247faf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1afe6aba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53c486a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38727bf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@451f9670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c5d9fd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b5bdd48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@705db00c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49c80461 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9aba98c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c1f946c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@115f8833 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18bd6d75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c8e4027 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c42112e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51b10885 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141635,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22941103 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ca5f20e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30c25bd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69ee9868 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bf0805f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58f63aad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a48de95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6778346c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@865a2a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61afd257 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5daf0481 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b52800c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e077d40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41a521bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@640bbe04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bb61069 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e3e8122 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67eed91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f3c98c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b2f212 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e1f4437 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f7c3000 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@345d7f93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141636,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@276c2968 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@43b6ff4e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4dc43b08 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123a64c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54db3882 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2063f1d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c9382a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dced3b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ce5a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f0e2944 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5812b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1470279d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73669160 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d6cc241 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@128eca86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d75b85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40fd378e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b588b1a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f66b80d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4250cf17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bb8c194 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e787a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e0be7e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141637,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ae7a742 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@177e75eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f2c3a31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3eb9e2c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39638fea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d567400 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23552029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@520b2929 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7732f05d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19aa6ccf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@137443bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18bd629d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@786dd9cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@326b1e19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3ea74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7069d915 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@665cc576 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cad8472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10512fd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e4efdb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb9ea7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e5775cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57e1b438 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141638,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9b4836 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f90a19a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a9c50b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@400667ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14085d60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7efeb20a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61f7cbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d740c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2362f041 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@288778c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55000bf5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a1eb0e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ecaab19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d18bf55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@689254a0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e468f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c0d897f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49763bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e5a10da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a5d1c48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a8b300d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@753e3063 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b4c22de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69c1daa3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141639,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ad994ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee01ede -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4400990e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23f357ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15bac8bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bd2323c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bda1ea1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@122f681a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@511385f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2baa0e38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50fe32fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f8ef47 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7489f1a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f29eace -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ee16033 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@509d207f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ba2977c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b9d7f5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ed862e4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37ff3598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48995d6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@83eacbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45bc301d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141640,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b218ec7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9d1c0f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74f7c638 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ef3c63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57bfc70f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@506e5b5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae28169 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3347ba72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae984aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38ffb826 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36988f99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38023712 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e98286c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2be39311 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59f535b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70663338 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f12be10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@685ed5f8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18762ded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59482228 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1360e52f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d0f03ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24bf4ea7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141641,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c7e717a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2823b247 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dc4c69e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d071f7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c6a3556 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@796e1716 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e5de93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ba48f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fe70d75 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16386f38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a052a2a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a28005b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d0324ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1817d3f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd0eeab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21d01d52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@752168f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@538fe5b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fa6ecd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e3d50f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a5d13e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cab36db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4403cfba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50ada501 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141642,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eafb20b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b475eeb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e33e26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cdb0cb6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f67d4cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ee67fcd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@163c7839 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@223678f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@732619ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ed31874 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bdb4627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a3b6270 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3990ccb4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f86634a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eddf0e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f542baa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@101eb798 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c8e0ae2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7db5ab78 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7332c61f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16f51d96 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ac0dcb0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a7e46fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141643,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f996f95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@713ed25d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@de7b596 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55f6634f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51974ba3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@563cc7f6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@664793a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d783e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6bece1c7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bdf983b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dfca61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a38deec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66d6f8b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@416fba14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e9cfc9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@127f8a1b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@433dcbb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fbe66d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@261f3d99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48de5413 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@65137efa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69174c26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36bb4768 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141644,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c7ff360 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72e5a8bb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2140bcbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d2c3ffa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ebb31cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61c4f6d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45450b01 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@693a3fbc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e684606 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@485c536d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@252c5464 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1dd358a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51476568 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7baa9316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ce26773 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bf32e05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51f57036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ee87b3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@519e426c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733fcd5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93177cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34bdd445 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17936d9c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a4b171f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141645,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18469484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42037435 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e700e64 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c410b5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@208f5ce2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7974261a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3739047e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3326c218 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6914e0d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31885f1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e6da7a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b5e9a70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582f1b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76012eaa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5572ef52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a407296 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78dd595e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d65c91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@451c2b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c2125b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ea49ec2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b236f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141646,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f41336a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a68979e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a04a9f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e9f62f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26155f3b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@688ff800 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ae9a33c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@357e7ac0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@399e07d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e46a718 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@198111af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f6fc025 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1686f484 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@de3ef2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cfe8b51 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3db10e03 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1cb2ffe5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e3ad1ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6676777a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5302bca1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@58c42cf3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a92bad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55314165 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141647,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@139018d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aca0de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2377a754 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c92e036 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f5711da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5548bcc6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b1bfc3c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20633639 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ed3a2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67530699 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@733dfc53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a56def3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@54f4b1dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bec1214 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37afac5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b725d1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af967f4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71ceae0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3bc93136 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@620a11fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@133d0c8e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17877622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6535b565 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141648,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5702ddce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ca0f2dc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bab70c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@486ca4e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9c1fc8d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@225cc97b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@770b18eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5aede155 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c4e4b11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189d46f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@377ed94a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22c61b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c14b42e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ecccd88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f39fb92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e797065 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77450e40 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fdd3dad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fe8aaaf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aa70b6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@754f2775 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53594c24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9be2932 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141649,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1bd0a719 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17a84f99 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e090924 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7606466e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68f0dfd8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3df191d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf7ff6b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f06f34d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5166f56a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60a8c1cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24606616 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c74754c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a5f180d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cdf1e30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b28856b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44794757 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d4b0a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62ee1ed1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6de2dc3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63af7d2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30ada866 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22273034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141650,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59df3b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41016b5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c321580 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78f30a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27aa2e1e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6dbbfa2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61296c0c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fdd5aab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f15d90 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ba9a00a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@714afe7f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b7faff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c92a24 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71f66f27 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5769de60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c32b670 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@420429d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49e9e8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19caad5f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60b21920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31ba607 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c520925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bef5a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141651,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b5978b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e402c04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@728703c8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@563c3e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59b61508 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76393236 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ceaf67a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@270d5ec4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4efa1178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@716d2213 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1adf4362 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5769c583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49066fc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51e3a14e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31c72c43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@289bb225 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@529998a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ebcaa6a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@161128ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63eeaab7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d2c52c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b858f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19f06794 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141652,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f5ddd0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10780452 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5443a97c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e95a8a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f41b1df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fcb7af1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@368a243d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b972db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7754fefa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53ec1e13 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34a0639f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a11e07a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a6278ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49cded35 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e46d308 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b71ee8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32bd4810 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14bfbf37 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ca9f03b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8f8509a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@505d745d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39f22f66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a49284b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@607fc118 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141653,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bbbbed6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@739a33d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10724690 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e1b2287 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@850709c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2af696bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28b213b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fea033e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@743548cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ae87b8a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@527ff2d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7da3f85b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@230eb410 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a93bfd0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27d76d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45d438e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13acba7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@456eae4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18f6236a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c811bac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a883108 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39baeb77 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@107e6259 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141654,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64d65781 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f1d2a62 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41928da5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@229d5364 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@742172e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32290835 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67835470 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2fcd2d97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5004a4bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a166628 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@574b97ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11aca925 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64f8c6e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d46cb70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76b8c6f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@229bc82 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2044ade8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b92ffdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b2d2316 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@142f595d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11019ddd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d11d401 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f967f26 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@311d3038 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141655,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cefbcb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fbdc4f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23beed6d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b7a85d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2aa1a42a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b9a77d1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9b264a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70607c7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@574706ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@fbbc76a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@91419a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ae41771 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15790ce7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@442809b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d58c980 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@375d14c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a1e28fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e7ff1aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4540312c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ae4ee7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a6641c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10452723 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5448ad56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141656,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@696236a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b562910 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62936166 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73a0427f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b8f9db9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@617b2b71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189afe69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69d8e4bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ecdde68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26d55e63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f38566 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47a069fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37eed1f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b44a721 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e69cd9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a428a1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4be2f3b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55813847 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55043a70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b0840af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17ab6d20 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b343c92 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f97583e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141657,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36936a5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a9dc3c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a7a0a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15745c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38183d6c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c8f5a48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1001598 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36a28497 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4fa82c72 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19a372d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@180e18ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3aaab106 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34efd19d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3658380f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e7b9d10 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@31b63558 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b9f79ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c70bef6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e612175 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e7dc914 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e2cb038 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a23805 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cfdb0d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141658,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@601a8769 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bdc4622 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da8254c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63c19247 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@57d46099 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e50c117 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@620e11d4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@db2e478 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@79168dff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4176b3b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53e49af8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@496f35d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45b207f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70c4f06c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70087d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3771b31b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a742552 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de4efe0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ed9e1bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40a6416 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f1f8f70 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@559ef29c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24bb259b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141659,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73c6cfce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4011bac9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62b9aa45 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f53dc74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a2ec8ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1724f691 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38e08db2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@395156a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@151498d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2534649e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1905f097 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ad26177 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b31a6fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da73192 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b90fbf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1270ee25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@460acc3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@206f367c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0c5ebb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@191888d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@189f298d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@419a2034 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f8d473a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@728e5c52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141660,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13bcc0cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@337c90c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46c6fac7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@46aa4281 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@33655823 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e2798ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fc65b2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6beca018 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72d15906 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40837f7e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@dd17b46 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3e684 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b52969d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20f17e06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26fc3a15 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3ae77f88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@556ef037 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7ec4a5d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@392f148d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38b5271f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1af88776 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41494bce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a416 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141661,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@111ff6d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f8210b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@441e25b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59201a30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37e7fe84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@683f4f7c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5db9cc67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@411ac7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c0a61a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e5c8bf0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d15a6a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4eeb90c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5bb6cf42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214fcccf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14f1ea8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7018f154 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d39fcca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42cf6216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b8c9eb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c295acd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e24282e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67423e2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38641d29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141662,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eddd7da -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@147e577 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7782083f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f17ceb3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7fe4b78c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4136895a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2612a2b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77219294 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d23132b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@582a338c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b9e0500 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10509fd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25df7cbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c608ddb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37cd2b52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2377a449 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d130e1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55b54a30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bfc6c6f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15b7c4b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@70b15bf1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@faa6b1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675de193 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141663,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4ce413a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f0a71f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a5ed05f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6287f5fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e928b14 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3de90d86 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a9fe4c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c5cc782 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@668c2936 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@137c9d5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dc95ae1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6733f20d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@60f1a244 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1790457b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20391b3a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@586226aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356c5c60 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c27c2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ca53c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1546b07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568cb811 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@659cb5cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f52aefd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dfcee43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141664,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e8399dd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e316b0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47e1572 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e4a7d48 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4c75462d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@297935bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dd86067 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3015f6a9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56b79aa8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3683990c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28c61ce3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37f917ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5df1a961 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7790102a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f293b3f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10901741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40fa0e93 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@371c47e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@114a72f9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f0558e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@116e4db5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44981b09 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68764d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141665,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5581da2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6638eb59 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@627ce4c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fe0f5fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a1056ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b530495 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e0c8dff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@669bd0c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e264785 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d729378 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb8e5c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f6e7b98 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7338099f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3a18bdd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@617a7da2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d8478ae -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e43f870 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@61ca384e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53efdeb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5da09d63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ca16edf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6c19354d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@412e3c68 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141666,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183cfaab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4f3cb8e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a2a4f12 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ad7f343 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25c1b310 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f10ccdd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@377a50f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52d2e0b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a2380cc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@735ccd71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16c4ee28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53d6630d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3515ac5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2037c8cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1133c74d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@711d006d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69721d61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@771dd122 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@789b6923 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@619c1035 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@187c9072 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f1c69e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@228065f2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141667,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@231cd42f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f1f2f3e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5055c775 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@715b9778 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@342398d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@126327b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cf324e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@413e8e52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@69f36bbd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62a6a7b9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@17075332 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@227e139e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50a22aef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7eec42d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2add9921 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@786d8d07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@395935b7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48e7511d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@eb5ddb1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1a246cd5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a2006ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@183ddd4b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38c5d3a8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141668,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13e42a1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b410973 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d6056d5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bae8e42 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7876eedc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b1f6a30 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4565178 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bf5132e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34ba1282 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5494b0cf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62fff6b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3140e5a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@558acf0e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24690fb8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@393e17d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f65a883 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c14e4de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@606c2e2b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29251d5e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@b268949 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@203faee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a3c1bd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@206df73a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141669,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1f3e4ebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42ab8329 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dba0ce8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@32a1e042 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f871b1f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@412b690d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cdbf08c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5145aea5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@367d1871 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c047cb2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fe8b62e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a413f0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dc42165 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@322a2b83 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@68a9b22e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b510758 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3782c45e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5497871e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@710a7760 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141698,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@52c2c5de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@73435bec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b048ded -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3917589a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dbf8653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@144a2e33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2b9ea2b2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28ceecd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c27a87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cbce23a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3269462d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6d9e8cc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5732c5b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c94505f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24342dce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3dd5753b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c0b307d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e3810b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bc96c4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@11d657a5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347b153a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a0e3167 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49678854 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a2f95a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141699,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6918f18 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d4f03a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@281f0571 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e8b0755 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34e8a6d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@119fc1c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@417b6a06 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27ac7cd3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@568398cb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@397d9a5b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23ccfc28 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67c25666 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d4ddf95 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78b72725 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3cdea9a7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49daa93c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@aa876e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b67f69a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2bdfdc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e3227e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5764659f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23c39acf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141700,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7d308a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41ff1390 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35c3cead -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ae93f38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d37a78f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@14ed522a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f7050bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c23a895 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6cb86341 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@123a10fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ff247b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e7435de -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6179f53a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2386b40b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e748021 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71418cf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fbf77c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@490fb6d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7340525b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e1b46eb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2abe3984 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1eb3ab2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f90e029 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3be8df2e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141701,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6041b3ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63a89e33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@bd1bbc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76bd043c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b8f16e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c715aa2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3af8d168 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@24fabeef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4721a4b0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@713352bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@d3cbe63 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cc074ed -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@360e4be1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d43592e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@35642b43 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4afc561e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@ee95de3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@49a4ef39 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d8eeb7d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75ea0a74 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3128476f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f9bab54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141702,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@756466d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@50f86bab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@23185de4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21db93d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@775f6b94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64e9e3e7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ff75e3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@784763d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7448b480 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f158735 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21bbb159 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62212762 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@537c7312 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20a36a67 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5cee2653 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@379f009b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74b0199 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c80a345 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1171c638 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e258755 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13d12c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@389e1328 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27414205 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141703,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@55ac7697 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6eddf9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74219e0f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@595dbbe4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ff37696 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b8ef84 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5d91ccda -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6927de17 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d945263 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f324a9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c16beba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b960a49 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44ae4812 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29a5bc87 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@30df3467 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bd0a477 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d089dc7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@494d5165 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@782827 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e75c743 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@779a9bef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@547f3a00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@158709b8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45989b94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141704,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66fe352b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b7cacce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@669ba6d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@675ed5ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@709ef301 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e2a3b00 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@653cab33 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74dee18d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@735f12ec -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c22c5bc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cc79c38 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f96d8a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74556dc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e3049db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5ed63079 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@174c80ff -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48a894f3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@544a62ac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b70c5a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4848f14d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44cbfbbb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4393be8c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e731ebf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141705,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71be4aa5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6a33710a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@340ad51e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@526aa3c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5102204c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20c8a047 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42883365 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@45fbdc61 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53b94d2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6014d6aa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6304d2d6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78871203 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@214394c5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6724a7c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@22f35b91 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@622442c6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f7c07fb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1d24b97a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25b3469c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6541129e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5abd8212 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@626a1f56 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5dd22221 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141706,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@306a0ee7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6f1a7ea -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@34d388cd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@785c675c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18a14386 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1364698f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b590ece -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@20a4e87e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@328b329f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2130d50 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71cf4ca2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76afbdfd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ddb767d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b8911b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c197712 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d27339 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41fc8520 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@28ff44d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4569b2e9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51b64ac6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@74c60055 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@996bf1c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141707,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@160ba3d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@62e320ad -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2663400c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab17583 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e0485ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6599389f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59c66700 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@48ca0cd6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72596920 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5e28fa34 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@343c6896 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2c086e7b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@15a112b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@c41a369 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b6e12af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@347ac40f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29d36cb9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2808d801 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3585cc5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6160ba1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5364a42c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d42a691 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c3d6b88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141708,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@18e99640 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e5a04b5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@460451c1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@517213a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fdd45a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b3c17fc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5fd2c303 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1efc2458 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75c86cd9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e90066b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@67931261 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1832342f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e2a2d54 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27d5a4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7f58dfdb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76c21f53 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@9fdcc52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@37c9a23c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e560519 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6888bb07 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f706518 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75d2ee88 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b1243e6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@66ef9900 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141709,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6abb7860 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@87f6d11 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@51ab028b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b16b627 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7a5faf5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@93b42e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8db614 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e548fac -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7976f27c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2a0993e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@47d9ada1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1daa6319 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@530197c2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@357188c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3f4f8f69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@71b39132 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e53a741 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@175d05d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@542af49d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39fdc52b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@a2714fa -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75aba3a3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76262f97 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141710,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@21a578e5 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@131728ef -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3b093795 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7c94fdf4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@767cd0d3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c06dea3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2ec1ff29 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@75e1dc58 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@658b607f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f1249ab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e0bd7d7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5573dd52 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6b45d71b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@760794c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64240b1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4d908466 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@8286fc0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@42c303d0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@561094f1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1fea99e1 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5477a3b3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@e70e76e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141711,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c154d4d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@177be2bf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7e78a1d8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@77ba62ee -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1b33671 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@499542e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72d85c69 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@56d1edce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e4949db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2d2890a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1da3aa7a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@44458b9d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4046155f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29023075 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29f2af55 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4183c095 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@311fd917 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@926d6d9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@359258af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6acab1e0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@25521acf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1afd80a2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3d0de472 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141712,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f71782 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3746e171 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@41c89fab -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@13b4dd94 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29c58367 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2908b247 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cb8450a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19d57574 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c8e0392 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@707810ce -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1e869bde -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2cca33af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@40343737 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@707aa19f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5a088c25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@27228b80 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@464e9ddc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@631dbad3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7dea7b31 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4e279187 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7cb545e2 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@72f0a385 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5c6d7d19 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141713,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eb474ba -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@78f8bedb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f63c6ca -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@38d1e78a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@354d458e -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@87e2b9a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@19bae9c9 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e60a0c3 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@715ee1b4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@88526fe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@463f7286 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6df2a43a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6ba7ab71 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4a34d22d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@772222af -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@f898b05 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@59a355a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@356a5105 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@533b2a5d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@10a9429c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@53643b4a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3461aa5a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3e021749 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141714,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@64651fb7 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@16a9c61a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2f238def -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1357f1d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4de1402b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1520c69b -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@26df7d25 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cb83374 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@4b1fcad0 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5818c936 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@573a145 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@489d110c -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7d610b85 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63ae2c66 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7b4fc0df -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b7aabcb -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2e32be02 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@3c6d6abe -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@39210cd4 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141715,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@378bccc8 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@cf2a695 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b1d152f -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6aebf534 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@36f892d -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1ab051db -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@29b79bcf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@1c408998 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@63995ddc -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@7bbfe6fd -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@6e4363a6 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@76d73bbf -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5f09ff04 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@5b056216 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@2eae6483 -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141716,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:39:01 pool-52-thread-1 WARN SnowflakeSinkServiceV1:82 - [SF_KAFKA_CONNECTOR] Cleaner encountered an exception class java.lang.IllegalStateException: -Snowpipe Streaming Doesnt Have Pipe Usage -[Ljava.lang.StackTraceElement;@616bb82a -17-08-2023 17:39:01 pool-52-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":1,"time":1692319141717,"error_number":"Snowpipe Streaming Doesnt Have Pipe Usage"} of type:kafka_fatal_error -17-08-2023 17:42:19 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe","start_time":1692319339194,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start -17-08-2023 17:42:19 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe_streaming","start_time":1692319339228,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start -17-08-2023 17:42:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:42:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:42:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:42:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 -17-08-2023 17:42:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8324131930661886881 -17-08-2023 17:42:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:42:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 -17-08-2023 17:42:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:40 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:42:40 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 17:42:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:42:40 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:42:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:42:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:40 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0, existing:[] -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 - service started -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8324131930661886881, topicPartition: test-0 -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8324131930661886881. -17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 retrieved 0 file names -17-08-2023 17:42:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881. -17-08-2023 17:42:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 doesn't exist -17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0 -17-08-2023 17:42:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 retrieved 0 file names -17-08-2023 17:42:41 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0: cleaner started -17-08-2023 17:42:41 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319360821} of type:kafka_pipe_start -17-08-2023 17:42:41 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz -17-08-2023 17:42:41 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:42:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881, filePath:TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz -17-08-2023 17:42:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881 -17-08-2023 17:42:42 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881, filePath:TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz -17-08-2023 17:42:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8324131930661886881/0/0_0_1692319361844.json.gz -17-08-2023 17:42:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:42:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:42:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:42:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:43:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:43:41 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0/insertReport?requestId=4b0f72c0-fc0d-4467-a85d-e02fc96a0c30 -17-08-2023 17:43:42 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:43:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573689508, X-Snowflake-Connection-Request: 324, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.340, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692317537861, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@3e569d78} -17-08-2023 17:43:42 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 -17-08-2023 17:43:42 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319360813,"end_time":1692319422485,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:43:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:43:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:43:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:43:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:44:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:44:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:44:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:44:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:44:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:44:42 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0/insertReport?requestId=d9a1de62-af0a-4ba6-8ae1-527a09dc1562&beginMark=1_-1 -17-08-2023 17:44:42 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:44:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574053822, X-Snowflake-Connection-Request: 728, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.198, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692317537861, X-Snowflake-Upstream: 10.181.19.142:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@55bc6db8} -17-08-2023 17:44:42 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 -17-08-2023 17:44:42 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8324131930661886881","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8324131930661886881","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8324131930661886881_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319422485,"end_time":1692319482892,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:44:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:44:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:44:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:45:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:45:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:45:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:45:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:45:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:45:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:45:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:45:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 -17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7418094664318925232 -17-08-2023 17:45:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 -17-08-2023 17:45:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:45:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:45:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 -17-08-2023 17:45:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:55 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:45:55 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 17:45:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:45:55 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:45:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:45:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:56 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0, existing:[] -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 - service started -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7418094664318925232, topicPartition: test-0 -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7418094664318925232. -17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 retrieved 0 file names -17-08-2023 17:45:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232. -17-08-2023 17:45:56 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 doesn't exist -17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0 -17-08-2023 17:45:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 retrieved 0 file names -17-08-2023 17:45:56 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0: cleaner started -17-08-2023 17:45:56 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319556026} of type:kafka_pipe_start -17-08-2023 17:45:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz -17-08-2023 17:45:56 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:45:57 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232, filePath:TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz -17-08-2023 17:45:57 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232 -17-08-2023 17:45:58 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232, filePath:TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz -17-08-2023 17:45:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7418094664318925232/0/0_0_1692319556974.json.gz -17-08-2023 17:45:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:46:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:46:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:46:56 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=2800bc44-1a23-483c-8f9a-034e04cc5727 -17-08-2023 17:46:57 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:46:57 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575220234, X-Snowflake-Connection-Request: 103, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.332, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.213:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@4b6bf39d} -17-08-2023 17:46:57 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 -17-08-2023 17:46:57 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319556018,"end_time":1692319617651,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:46:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:47:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:25 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:47:30 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:47:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:47:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:47:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:47:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:47:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:47:57 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=6891ecd1-f9fd-44e3-a6bf-b7f01ccb199c&beginMark=1_-1 -17-08-2023 17:47:58 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:47:58 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573760515, X-Snowflake-Connection-Request: 63, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.198, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.213:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@4e48ba7a} -17-08-2023 17:47:58 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 -17-08-2023 17:47:58 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319617651,"end_time":1692319678380,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:47:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:48:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:48:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:48:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:48:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:48:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:48:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:48:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:48:58 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0/insertReport?requestId=40112fc0-52fc-4c12-99a2-affbdaeeecd2&beginMark=1_-1 -17-08-2023 17:48:59 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:48:59 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574402908, X-Snowflake-Connection-Request: 214, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.167, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319511159, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@640eb580} -17-08-2023 17:48:59 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 0 -17-08-2023 17:48:59 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7418094664318925232","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7418094664318925232","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7418094664318925232_0","processed-offset":0,"flushed-offset":0,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319678380,"end_time":1692319739360,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:49:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:49:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:49:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:49:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:49:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:49:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:50:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:50:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 -17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_179094934082246481 -17-08-2023 17:50:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 -17-08-2023 17:50:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:50:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:50:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 -17-08-2023 17:50:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:35 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:50:35 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 17:50:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:50:35 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:50:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:50:35 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:35 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, existing:[] -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 - service started -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_179094934082246481, topicPartition: test-0 -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_179094934082246481. -17-08-2023 17:50:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 retrieved 0 file names -17-08-2023 17:50:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481. -17-08-2023 17:50:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 doesn't exist -17-08-2023 17:50:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:50:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 retrieved 0 file names -17-08-2023 17:50:36 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: cleaner started -17-08-2023 17:50:36 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_179094934082246481","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319835165} of type:kafka_pipe_start -17-08-2023 17:50:36 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz -17-08-2023 17:50:36 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:50:36 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481, filePath:TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz -17-08-2023 17:50:36 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 -17-08-2023 17:50:36 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481, filePath:TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz -17-08-2023 17:50:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz -17-08-2023 17:50:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz] -17-08-2023 17:50:36 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_179094934082246481/0/0_0_1692319836135.json.gz] -17-08-2023 17:50:36 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0/insertFiles?requestId=130ef0ad-56aa-4625-bbe8-abb98094941d&showSkippedFiles=false -17-08-2023 17:50:37 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:50:37 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574054195, X-Snowflake-Connection-Request: 77, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.752, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.19.142:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:50:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:50:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:50:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:50:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:50:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: cleaner terminated -17-08-2023 17:51:08 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:51:08 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:51:08 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:51:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:08 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_179094934082246481","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1734,"average_commit_lag_file_count":1,"start_time":1692319835156,"end_time":1692319868680,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:51:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0: service closed -17-08-2023 17:51:08 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 -17-08-2023 17:51:08 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 is:0, names:[] -17-08-2023 17:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_179094934082246481 dropped -17-08-2023 17:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_179094934082246481_0 dropped -17-08-2023 17:51:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 -17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5989379599178019927 -17-08-2023 17:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 -17-08-2023 17:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:09 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:51:09 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:51:09 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:51:09 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:51:09 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:09 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, existing:[] -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 - service started -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5989379599178019927, topicPartition: test-0 -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:51:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 17:51:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:09 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5989379599178019927. -17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 retrieved 0 file names -17-08-2023 17:51:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927. -17-08-2023 17:51:10 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 doesn't exist -17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 retrieved 0 file names -17-08-2023 17:51:10 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: cleaner started -17-08-2023 17:51:10 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5989379599178019927","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319869819} of type:kafka_pipe_start -17-08-2023 17:51:10 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz -17-08-2023 17:51:10 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 17:51:10 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927, filePath:TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz -17-08-2023 17:51:11 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 -17-08-2023 17:51:11 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927, filePath:TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz -17-08-2023 17:51:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz -17-08-2023 17:51:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz] -17-08-2023 17:51:11 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_5989379599178019927/0/1_1_1692319870794.json.gz] -17-08-2023 17:51:11 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0/insertFiles?requestId=949760a0-5805-46b0-95a3-8494d05e042a&showSkippedFiles=false -17-08-2023 17:51:12 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:51:12 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574499316, X-Snowflake-Connection-Request: 124, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.621, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: cleaner terminated -17-08-2023 17:51:43 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:51:43 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:51:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:51:43 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5989379599178019927","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1405,"average_commit_lag_file_count":1,"start_time":1692319869816,"end_time":1692319903333,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:51:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0: service closed -17-08-2023 17:51:43 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 -17-08-2023 17:51:43 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 is:0, names:[] -17-08-2023 17:51:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5989379599178019927 dropped -17-08-2023 17:51:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5989379599178019927_0 dropped -17-08-2023 17:51:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 -17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8129691469536046428 -17-08-2023 17:51:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 -17-08-2023 17:51:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:44 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:51:44 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:51:44 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:51:44 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:51:44 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:44 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, existing:[] -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 - service started -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8129691469536046428, topicPartition: test-0 -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8129691469536046428. -17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 retrieved 0 file names -17-08-2023 17:51:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428. -17-08-2023 17:51:45 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 doesn't exist -17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:51:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 retrieved 0 file names -17-08-2023 17:51:45 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8129691469536046428","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319904503} of type:kafka_pipe_start -17-08-2023 17:51:45 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: cleaner started -17-08-2023 17:51:45 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz -17-08-2023 17:51:45 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:51:45 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428, filePath:TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz -17-08-2023 17:51:45 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 -17-08-2023 17:51:46 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428, filePath:TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz -17-08-2023 17:51:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz -17-08-2023 17:51:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz] -17-08-2023 17:51:46 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8129691469536046428/0/0_0_1692319905442.json.gz] -17-08-2023 17:51:46 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0/insertFiles?requestId=5576c06e-fdf0-4832-8c7f-1a95bdcb4103&showSkippedFiles=false -17-08-2023 17:51:46 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:51:46 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573914218, X-Snowflake-Connection-Request: 209, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.473, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319734055, X-Snowflake-Upstream: 10.181.21.27:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:51:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:51:49 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:54 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:51:59 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:52:04 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:52:09 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:52:14 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:52:17 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: cleaner terminated -17-08-2023 17:52:17 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:52:17 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:52:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:52:17 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:52:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8129691469536046428","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1295,"average_commit_lag_file_count":1,"start_time":1692319904502,"end_time":1692319937851,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:52:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0: service closed -17-08-2023 17:52:17 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 -17-08-2023 17:52:17 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 is:0, names:[] -17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8129691469536046428 dropped -17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8129691469536046428_0 dropped -17-08-2023 17:52:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:52:18 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:18 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:52:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:19 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:52:19 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:52:19 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:52:19 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:52:19 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:19 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0, existing:[] -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 - service started -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6867286562641800447, topicPartition: test-0 -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:52:19 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: unknown bytes -17-08-2023 17:52:19 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 17:52:19 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6867286562641800447. -17-08-2023 17:52:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 retrieved 0 file names -17-08-2023 17:52:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447. -17-08-2023 17:52:19 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 doesn't exist -17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0 -17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447 retrieved 0 file names -17-08-2023 17:52:20 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0: cleaner started -17-08-2023 17:52:20 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692319939224} of type:kafka_pipe_start -17-08-2023 17:52:20 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_key_1692319940175.gz -17-08-2023 17:52:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_key_1692319940175.gz to table stage: kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:20 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_value_1692319940790.gz -17-08-2023 17:52:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_6867286562641800447/0/1_value_1692319940790.gz to table stage: kafka_connector_test_table_6867286562641800447 -17-08-2023 17:52:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:49 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:52:54 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:52:59 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:20 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692319939221,"end_time":1692320000180,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:53:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:53:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:49 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:52 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:53:54 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:53:59 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:54:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:54:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:54:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:54:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:54:20 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6867286562641800447","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6867286562641800447","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6867286562641800447_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320000180,"end_time":1692320060187,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:57:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:57:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:57:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:57:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 -17-08-2023 17:57:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_28885932869605191 -17-08-2023 17:57:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:57:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 -17-08-2023 17:57:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:31 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:57:31 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 17:57:31 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:57:31 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:57:31 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:57:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:31 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, existing:[] -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 - service started -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_28885932869605191, topicPartition: test-0 -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_28885932869605191. -17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 retrieved 0 file names -17-08-2023 17:57:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191. -17-08-2023 17:57:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 doesn't exist -17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:57:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 retrieved 0 file names -17-08-2023 17:57:32 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: cleaner started -17-08-2023 17:57:32 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_28885932869605191","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320251712} of type:kafka_pipe_start -17-08-2023 17:57:32 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz -17-08-2023 17:57:32 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:57:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191, filePath:TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz -17-08-2023 17:57:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 -17-08-2023 17:57:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191, filePath:TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz -17-08-2023 17:57:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz -17-08-2023 17:57:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz] -17-08-2023 17:57:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_28885932869605191/0/0_0_1692320252681.json.gz] -17-08-2023 17:57:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0/insertFiles?requestId=e55c4bcd-ed7a-45b3-91da-fa7bfb413926&showSkippedFiles=false -17-08-2023 17:57:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:57:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574630358, X-Snowflake-Connection-Request: 344, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.88, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.594, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.19.183:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:57:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:57:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:57:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:57:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:57:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:57:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:06 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:58:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: cleaner terminated -17-08-2023 17:58:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:58:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:58:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_28885932869605191","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2151,"average_commit_lag_file_count":1,"start_time":1692320251699,"end_time":1692320286060,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:58:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0: service closed -17-08-2023 17:58:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 -17-08-2023 17:58:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 is:0, names:[] -17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_28885932869605191 dropped -17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_28885932869605191_0 dropped -17-08-2023 17:58:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 -17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4067068134490256806 -17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 -17-08-2023 17:58:07 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:07 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:58:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:58:07 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:58:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:58:07 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:07 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, existing:[] -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 - service started -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4067068134490256806, topicPartition: test-0 -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:58:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 17:58:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4067068134490256806. -17-08-2023 17:58:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 retrieved 0 file names -17-08-2023 17:58:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806. -17-08-2023 17:58:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 doesn't exist -17-08-2023 17:58:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 retrieved 0 file names -17-08-2023 17:58:08 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320287478} of type:kafka_pipe_start -17-08-2023 17:58:08 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz -17-08-2023 17:58:08 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 17:58:08 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: cleaner started -17-08-2023 17:58:08 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","processed-offset":1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":178,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320287476,"end_time":1692320288397,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 17:58:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806, filePath:TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz -17-08-2023 17:58:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 -17-08-2023 17:58:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806, filePath:TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz -17-08-2023 17:58:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz -17-08-2023 17:58:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz] -17-08-2023 17:58:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4067068134490256806/0/1_1_1692320288393.json.gz] -17-08-2023 17:58:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0/insertFiles?requestId=773417e7-d42f-46f2-b9e8-b9977c7a23c3&showSkippedFiles=false -17-08-2023 17:58:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:58:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 573955694, X-Snowflake-Connection-Request: 128, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.387, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.20.237:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:58:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:12 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:17 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:22 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:27 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:32 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:37 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: cleaner terminated -17-08-2023 17:58:40 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:58:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:58:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:58:40 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4067068134490256806","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1204,"average_commit_lag_file_count":1,"start_time":1692320288397,"end_time":1692320320536,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:58:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0: service closed -17-08-2023 17:58:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 -17-08-2023 17:58:40 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 is:0, names:[] -17-08-2023 17:58:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4067068134490256806 dropped -17-08-2023 17:58:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4067068134490256806_0 dropped -17-08-2023 17:58:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 -17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4898200611066290160 -17-08-2023 17:58:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 -17-08-2023 17:58:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:58:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:58:41 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:58:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:58:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, existing:[] -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 - service started -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4898200611066290160, topicPartition: test-0 -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4898200611066290160. -17-08-2023 17:58:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 retrieved 0 file names -17-08-2023 17:58:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160. -17-08-2023 17:58:42 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 doesn't exist -17-08-2023 17:58:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:58:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 retrieved 0 file names -17-08-2023 17:58:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4898200611066290160","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320321795} of type:kafka_pipe_start -17-08-2023 17:58:43 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: cleaner started -17-08-2023 17:58:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz -17-08-2023 17:58:43 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 17:58:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160, filePath:TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz -17-08-2023 17:58:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 -17-08-2023 17:58:43 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160, filePath:TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz -17-08-2023 17:58:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz -17-08-2023 17:58:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz] -17-08-2023 17:58:43 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4898200611066290160/0/0_0_1692320323084.json.gz] -17-08-2023 17:58:43 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0/insertFiles?requestId=0da7d33f-b5d5-448b-9e43-a6ae4f502335&showSkippedFiles=false -17-08-2023 17:58:44 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 00:58:44 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574713348, X-Snowflake-Connection-Request: 1, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.410, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692319920359, X-Snowflake-Upstream: 10.181.19.132:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 17:58:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:58:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:58:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:59:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:59:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:59:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:59:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: cleaner terminated -17-08-2023 17:59:15 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 17:59:15 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 17:59:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 17:59:15 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 17:59:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4898200611066290160","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1114,"average_commit_lag_file_count":1,"start_time":1692320321794,"end_time":1692320355099,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 17:59:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0: service closed -17-08-2023 17:59:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 -17-08-2023 17:59:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 is:0, names:[] -17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4898200611066290160 dropped -17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4898200611066290160_0 dropped -17-08-2023 17:59:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 17:59:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:16 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 17:59:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 17:59:16 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 17:59:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 17:59:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0, existing:[] -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 - service started -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4032550198220204688, topicPartition: test-0 -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 17:59:16 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: unknown bytes -17-08-2023 17:59:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 17:59:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4032550198220204688. -17-08-2023 17:59:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 retrieved 0 file names -17-08-2023 17:59:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688. -17-08-2023 17:59:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 doesn't exist -17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0 -17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688 retrieved 0 file names -17-08-2023 17:59:17 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0: cleaner started -17-08-2023 17:59:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4032550198220204688","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320356510} of type:kafka_pipe_start -17-08-2023 17:59:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_key_1692320357310.gz -17-08-2023 17:59:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_key_1692320357310.gz to table stage: kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_value_1692320357870.gz -17-08-2023 17:59:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4032550198220204688/0/1_value_1692320357870.gz to table stage: kafka_connector_test_table_4032550198220204688 -17-08-2023 17:59:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:36 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:41 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:46 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 17:59:51 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 17:59:56 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:01 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:06 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:17 pool-13-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4032550198220204688","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4032550198220204688","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4032550198220204688_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320356506,"end_time":1692320417323,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 18:00:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:00:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -17-08-2023 18:00:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:00:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 -17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8459553152922049676 -17-08-2023 18:00:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 -17-08-2023 18:00:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:00:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:00:44 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 -17-08-2023 18:00:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:44 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:00:45 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 18:00:45 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:00:45 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:00:45 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:00:45 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:45 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0, existing:[] -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 - service started -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8459553152922049676, topicPartition: test-0 -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8459553152922049676. -17-08-2023 18:00:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 retrieved 0 file names -17-08-2023 18:00:45 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676. -17-08-2023 18:00:45 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 doesn't exist -17-08-2023 18:00:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0 -17-08-2023 18:00:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676 retrieved 0 file names -17-08-2023 18:00:46 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0: cleaner started -17-08-2023 18:00:46 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8459553152922049676","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8459553152922049676","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8459553152922049676_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320445097} of type:kafka_pipe_start -17-08-2023 18:00:46 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8459553152922049676/0/0_0_1692320446358.json.gz -17-08-2023 18:00:46 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:01:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:01:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 -17-08-2023 18:01:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4764356504904401298 -17-08-2023 18:01:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:01:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 -17-08-2023 18:01:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:13 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:01:13 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 18:01:13 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:01:13 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:01:13 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:01:13 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:13 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, existing:[] -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 - service started -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4764356504904401298, topicPartition: test-0 -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4764356504904401298. -17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 retrieved 0 file names -17-08-2023 18:01:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298. -17-08-2023 18:01:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 doesn't exist -17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 retrieved 0 file names -17-08-2023 18:01:14 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: cleaner started -17-08-2023 18:01:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4764356504904401298","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320473586} of type:kafka_pipe_start -17-08-2023 18:01:14 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz -17-08-2023 18:01:14 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:01:14 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298, filePath:TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz -17-08-2023 18:01:14 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 -17-08-2023 18:01:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298, filePath:TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz -17-08-2023 18:01:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz -17-08-2023 18:01:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz] -17-08-2023 18:01:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4764356504904401298/0/0_0_1692320474415.json.gz] -17-08-2023 18:01:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0/insertFiles?requestId=a3a4e95a-ace1-4ade-88ac-ef19de60009f&showSkippedFiles=false -17-08-2023 18:01:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:01:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574194690, X-Snowflake-Connection-Request: 92, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.510, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.22.235:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:01:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:18 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: cleaner terminated -17-08-2023 18:01:46 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:01:46 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:01:46 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:01:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:46 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4764356504904401298","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1682,"average_commit_lag_file_count":1,"start_time":1692320473578,"end_time":1692320506918,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:01:46 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0: service closed -17-08-2023 18:01:46 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 -17-08-2023 18:01:46 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 is:0, names:[] -17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4764356504904401298 dropped -17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4764356504904401298_0 dropped -17-08-2023 18:01:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:01:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 -17-08-2023 18:01:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3495569082225838574 -17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:01:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 -17-08-2023 18:01:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:48 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:01:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:01:48 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:01:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:01:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:48 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, existing:[] -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 - service started -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3495569082225838574, topicPartition: test-0 -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:01:48 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 18:01:48 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3495569082225838574. -17-08-2023 18:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 retrieved 0 file names -17-08-2023 18:01:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574. -17-08-2023 18:01:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 doesn't exist -17-08-2023 18:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 retrieved 0 file names -17-08-2023 18:01:49 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320508306} of type:kafka_pipe_start -17-08-2023 18:01:49 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz -17-08-2023 18:01:49 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 18:01:49 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: cleaner started -17-08-2023 18:01:49 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","processed-offset":1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":178,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692320508303,"end_time":1692320509267,"is_pipe_closing":false} of type:kafka_pipe_usage -17-08-2023 18:01:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574, filePath:TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz -17-08-2023 18:01:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 -17-08-2023 18:01:49 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574, filePath:TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz -17-08-2023 18:01:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz -17-08-2023 18:01:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz] -17-08-2023 18:01:49 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3495569082225838574/0/1_1_1692320509259.json.gz] -17-08-2023 18:01:49 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0/insertFiles?requestId=e9ab114d-2391-4dbc-97e9-2bef7d42acaa&showSkippedFiles=false -17-08-2023 18:01:51 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:01:51 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574083877, X-Snowflake-Connection-Request: 114, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.555, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:01:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:01:53 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:01:58 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: cleaner terminated -17-08-2023 18:02:22 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:02:22 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:02:22 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:02:22 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3495569082225838574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2436,"average_commit_lag_file_count":1,"start_time":1692320509267,"end_time":1692320542991,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:02:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0: service closed -17-08-2023 18:02:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 -17-08-2023 18:02:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 is:0, names:[] -17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3495569082225838574 dropped -17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3495569082225838574_0 dropped -17-08-2023 18:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 -17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 -17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7401106389637088935 -17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 -17-08-2023 18:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 -17-08-2023 18:02:24 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:02:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:02:24 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:02:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:02:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 -17-08-2023 18:02:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0, existing:[] -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 - service started -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7401106389637088935, topicPartition: test-0 -17-08-2023 18:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7401106389637088935 dropped -17-08-2023 18:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7401106389637088935_0 dropped -17-08-2023 18:02:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 -17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 -17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6415826046470952295 -17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 -17-08-2023 18:02:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 -17-08-2023 18:02:25 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:02:25 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:02:25 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:02:25 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 -17-08-2023 18:02:25 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0, existing:[] -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 - service started -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6415826046470952295, topicPartition: test-0 -17-08-2023 18:02:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6415826046470952295 dropped -17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6415826046470952295_0 dropped -17-08-2023 18:02:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:02:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 -17-08-2023 18:02:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6646659250009994864 -17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:02:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 -17-08-2023 18:02:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:27 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:02:27 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:02:27 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:02:27 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:27 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, existing:[] -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 - service started -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6646659250009994864, topicPartition: test-0 -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6646659250009994864. -17-08-2023 18:02:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 retrieved 0 file names -17-08-2023 18:02:27 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864. -17-08-2023 18:02:27 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 doesn't exist -17-08-2023 18:02:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:02:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 retrieved 0 file names -17-08-2023 18:02:28 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: cleaner started -17-08-2023 18:02:28 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6646659250009994864","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320547237} of type:kafka_pipe_start -17-08-2023 18:02:28 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz -17-08-2023 18:02:28 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:02:28 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864, filePath:TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz -17-08-2023 18:02:28 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 -17-08-2023 18:02:29 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864, filePath:TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz -17-08-2023 18:02:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz -17-08-2023 18:02:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz] -17-08-2023 18:02:29 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6646659250009994864/0/0_0_1692320548377.json.gz] -17-08-2023 18:02:29 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0/insertFiles?requestId=bd47c693-89d9-4828-abc3-98b494db16f3&showSkippedFiles=false -17-08-2023 18:02:29 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 1; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 1; pending: 0; available: 0; max: 100] - -17-08-2023 18:02:29 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:02:29 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574786314, X-Snowflake-Connection-Request: 79, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.554, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:02:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:02:34 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:39 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:44 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:49 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:54 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:02:59 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: cleaner terminated -17-08-2023 18:03:00 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:03:00 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:03:00 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:03:00 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6646659250009994864","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1331,"average_commit_lag_file_count":1,"start_time":1692320547236,"end_time":1692320580786,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:03:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0: service closed -17-08-2023 18:03:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 -17-08-2023 18:03:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 is:0, names:[] -17-08-2023 18:03:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6646659250009994864 dropped -17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6646659250009994864_0 dropped -17-08-2023 18:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 -17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3951058095210931105 -17-08-2023 18:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 -17-08-2023 18:03:01 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:01 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:03:01 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:01 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:01 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:03:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:01 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, existing:[] -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 - service started -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3951058095210931105, topicPartition: test-0 -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:03:01 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:03:01 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:03:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:02 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3951058095210931105. -17-08-2023 18:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 retrieved 0 file names -17-08-2023 18:03:02 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105. -17-08-2023 18:03:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 doesn't exist -17-08-2023 18:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 retrieved 0 file names -17-08-2023 18:03:03 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: cleaner started -17-08-2023 18:03:03 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3951058095210931105","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692320581974} of type:kafka_pipe_start -17-08-2023 18:03:03 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz -17-08-2023 18:03:03 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 -17-08-2023 18:03:03 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105, filePath:TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz -17-08-2023 18:03:03 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 -17-08-2023 18:03:03 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105, filePath:TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz -17-08-2023 18:03:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz -17-08-2023 18:03:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz] -17-08-2023 18:03:03 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3951058095210931105/0/1_1_1692320583135.json.gz] -17-08-2023 18:03:03 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0/insertFiles?requestId=377b4a1a-63ff-457a-a3ee-1316e3635a9c&showSkippedFiles=false -17-08-2023 18:03:04 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:03:04 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574388491, X-Snowflake-Connection-Request: 65, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.470, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320418077, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:03:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:06 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: cleaner terminated -17-08-2023 18:03:35 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:03:35 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:03:35 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:03:35 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:35 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3951058095210931105","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1309,"average_commit_lag_file_count":1,"start_time":1692320581973,"end_time":1692320615528,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:03:35 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0: service closed -17-08-2023 18:03:35 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 -17-08-2023 18:03:35 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 is:0, names:[] -17-08-2023 18:03:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3951058095210931105 dropped -17-08-2023 18:03:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3951058095210931105_0 dropped -17-08-2023 18:03:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 -17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 -17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2747719583120231178 -17-08-2023 18:03:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 -17-08-2023 18:03:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 -17-08-2023 18:03:36 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:36 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:03:36 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:36 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:03:36 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 -17-08-2023 18:03:36 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0, existing:[] -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 - service started -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2747719583120231178, topicPartition: test-0 -17-08-2023 18:03:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2747719583120231178 dropped -17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2747719583120231178_0 dropped -17-08-2023 18:03:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 -17-08-2023 18:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8941582472149256418 -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 -17-08-2023 18:03:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 -17-08-2023 18:03:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:38 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:03:38 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 -17-08-2023 18:03:38 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0, existing:[] -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 - service started -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8941582472149256418, topicPartition: test-0 -17-08-2023 18:03:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8941582472149256418 dropped -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8941582472149256418_0 dropped -17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:38 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:38 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:03:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:03:38 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:38 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:03:38 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:03:38 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:03:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -17-08-2023 18:03:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:03:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:03:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:03:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:03:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3921686411842415796 doesn't exist -17-08-2023 18:03:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3921686411842415796. -17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3921686411842415796 -17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:03:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:03:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:39 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:03:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:03:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:03:40 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -17-08-2023 18:03:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:03:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:03:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:03:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:03:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1041 -17-08-2023 18:03:40 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4613830906384878297 doesn't exist -17-08-2023 18:03:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4613830906384878297. -17-08-2023 18:03:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4613830906384878297 -17-08-2023 18:03:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -17-08-2023 18:03:41 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:03:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:41 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:03:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:03:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:03:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1041 -17-08-2023 18:03:41 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 -17-08-2023 18:03:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:03:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:03:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:03:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:03:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1645110627355484471 doesn't exist -17-08-2023 18:03:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1645110627355484471. -17-08-2023 18:03:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1645110627355484471 -17-08-2023 18:03:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:41 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:42 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:03:42 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:03:42 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:03:42 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:03:42 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:03:42 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:03:42 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:03:42 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1042 -17-08-2023 18:03:42 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 -17-08-2023 18:03:42 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:03:42 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:03:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:03:42 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:03:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:03:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1044 -17-08-2023 18:03:42 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_5970348261965741519 doesn't exist -17-08-2023 18:03:42 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_5970348261965741519. -17-08-2023 18:03:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5970348261965741519 -17-08-2023 18:03:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:10:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:10:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:10:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:10:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 -17-08-2023 18:10:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8312607821312555903 -17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:10:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 -17-08-2023 18:10:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:10:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 18:10:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:10:32 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:10:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:10:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, existing:[] -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 - service started -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8312607821312555903, topicPartition: test-0 -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8312607821312555903. -17-08-2023 18:10:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 retrieved 0 file names -17-08-2023 18:10:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903. -17-08-2023 18:10:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 doesn't exist -17-08-2023 18:10:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:10:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 retrieved 0 file names -17-08-2023 18:10:33 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: cleaner started -17-08-2023 18:10:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8312607821312555903","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321032333} of type:kafka_pipe_start -17-08-2023 18:10:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz -17-08-2023 18:10:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:10:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903, filePath:TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz -17-08-2023 18:10:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 -17-08-2023 18:10:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903, filePath:TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz -17-08-2023 18:10:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz -17-08-2023 18:10:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz] -17-08-2023 18:10:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8312607821312555903/0/0_0_1692321033331.json.gz] -17-08-2023 18:10:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0/insertFiles?requestId=541da82e-1ae0-405f-ba62-be14c8674438&showSkippedFiles=false -17-08-2023 18:10:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:10:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575787957, X-Snowflake-Connection-Request: 448, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.497, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.12:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:10:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:10:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:10:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:10:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:10:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:10:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: cleaner terminated -17-08-2023 18:11:05 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:11:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:11:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:11:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8312607821312555903","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1537,"average_commit_lag_file_count":1,"start_time":1692321032326,"end_time":1692321065756,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:11:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0: service closed -17-08-2023 18:11:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 -17-08-2023 18:11:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 is:0, names:[] -17-08-2023 18:11:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8312607821312555903 dropped -17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8312607821312555903_0 dropped -17-08-2023 18:11:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 -17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7267408819954433042 -17-08-2023 18:11:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 -17-08-2023 18:11:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:06 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:11:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:11:06 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:11:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:11:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, existing:[] -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 - service started -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7267408819954433042, topicPartition: test-0 -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:11:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -17-08-2023 18:11:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:11:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7267408819954433042. -17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 retrieved 0 file names -17-08-2023 18:11:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042. -17-08-2023 18:11:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 doesn't exist -17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 retrieved 0 file names -17-08-2023 18:11:07 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: cleaner started -17-08-2023 18:11:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7267408819954433042","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321066915} of type:kafka_pipe_start -17-08-2023 18:11:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz -17-08-2023 18:11:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 18:11:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042, filePath:TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz -17-08-2023 18:11:08 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 -17-08-2023 18:11:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042, filePath:TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz -17-08-2023 18:11:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz -17-08-2023 18:11:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz] -17-08-2023 18:11:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7267408819954433042/0/1_1_1692321067806.json.gz] -17-08-2023 18:11:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0/insertFiles?requestId=e3d6c9d1-9a64-4c06-911a-cb696bf333b9&showSkippedFiles=false -17-08-2023 18:11:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:11:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574287154, X-Snowflake-Connection-Request: 170, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.364, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.119:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:11:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:26 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:36 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: cleaner terminated -17-08-2023 18:11:39 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:11:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:11:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:11:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7267408819954433042","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1270,"average_commit_lag_file_count":1,"start_time":1692321066913,"end_time":1692321099962,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:11:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0: service closed -17-08-2023 18:11:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 -17-08-2023 18:11:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 is:0, names:[] -17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7267408819954433042 dropped -17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7267408819954433042_0 dropped -17-08-2023 18:11:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:11:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 -17-08-2023 18:11:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1282015380187074791 -17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:11:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 -17-08-2023 18:11:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:11:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:11:41 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:11:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:11:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, existing:[] -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 - service started -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1282015380187074791, topicPartition: test-0 -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1282015380187074791. -17-08-2023 18:11:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 retrieved 0 file names -17-08-2023 18:11:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791. -17-08-2023 18:11:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 doesn't exist -17-08-2023 18:11:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:11:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 retrieved 0 file names -17-08-2023 18:11:42 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: cleaner started -17-08-2023 18:11:42 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1282015380187074791","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321101234} of type:kafka_pipe_start -17-08-2023 18:11:42 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz -17-08-2023 18:11:42 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:11:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791, filePath:TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz -17-08-2023 18:11:42 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 -17-08-2023 18:11:42 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791, filePath:TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz -17-08-2023 18:11:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz -17-08-2023 18:11:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz] -17-08-2023 18:11:42 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1282015380187074791/0/0_0_1692321102137.json.gz] -17-08-2023 18:11:42 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0/insertFiles?requestId=df06f6d4-2b62-4bfc-b7ba-2069e9db3a93&showSkippedFiles=false -17-08-2023 18:11:43 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:11:43 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574527671, X-Snowflake-Connection-Request: 159, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.40, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.428, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.156:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:11:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:11:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:11:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: cleaner terminated -17-08-2023 18:12:14 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:12:14 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:12:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:12:14 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1282015380187074791","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1115,"average_commit_lag_file_count":1,"start_time":1692321101230,"end_time":1692321134049,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:12:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0: service closed -17-08-2023 18:12:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 -17-08-2023 18:12:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 is:0, names:[] -17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1282015380187074791 dropped -17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1282015380187074791_0 dropped -17-08-2023 18:12:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:12:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 -17-08-2023 18:12:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6790576732472386043 -17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:12:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 -17-08-2023 18:12:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:15 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:12:15 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:12:15 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:12:15 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:12:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:15 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, existing:[] -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 - service started -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6790576732472386043, topicPartition: test-0 -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:12:15 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:12:15 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6790576732472386043. -17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 retrieved 0 file names -17-08-2023 18:12:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043. -17-08-2023 18:12:15 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 doesn't exist -17-08-2023 18:12:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 retrieved 0 file names -17-08-2023 18:12:16 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6790576732472386043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321135261} of type:kafka_pipe_start -17-08-2023 18:12:16 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: cleaner started -17-08-2023 18:12:16 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz -17-08-2023 18:12:16 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 18:12:16 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043, filePath:TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz -17-08-2023 18:12:16 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 -17-08-2023 18:12:16 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043, filePath:TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz -17-08-2023 18:12:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz -17-08-2023 18:12:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz] -17-08-2023 18:12:16 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6790576732472386043/0/1_1_1692321136089.json.gz] -17-08-2023 18:12:16 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0/insertFiles?requestId=20398aef-d45c-4766-934d-3e29876e90ec&showSkippedFiles=false -17-08-2023 18:12:17 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:12:17 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574456708, X-Snowflake-Connection-Request: 66, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.143, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.508, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.12:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:12:17 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:20 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:25 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:30 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:35 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:40 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:45 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: cleaner terminated -17-08-2023 18:12:48 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:12:48 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:12:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:12:48 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:48 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6790576732472386043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1249,"average_commit_lag_file_count":1,"start_time":1692321135238,"end_time":1692321168243,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:12:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0: service closed -17-08-2023 18:12:48 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 -17-08-2023 18:12:48 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 is:0, names:[] -17-08-2023 18:12:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6790576732472386043 dropped -17-08-2023 18:12:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6790576732472386043_0 dropped -17-08-2023 18:12:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 -17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:49 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2269349621436228672 -17-08-2023 18:12:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 -17-08-2023 18:12:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:49 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:12:49 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:12:49 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:12:49 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:12:49 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:49 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, existing:[] -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 - service started -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2269349621436228672, topicPartition: test-0 -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2269349621436228672. -17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 retrieved 0 file names -17-08-2023 18:12:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672. -17-08-2023 18:12:50 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 doesn't exist -17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:12:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 retrieved 0 file names -17-08-2023 18:12:50 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: cleaner started -17-08-2023 18:12:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2269349621436228672","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321169653} of type:kafka_pipe_start -17-08-2023 18:12:50 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz -17-08-2023 18:12:50 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:12:50 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672, filePath:TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz -17-08-2023 18:12:50 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 -17-08-2023 18:12:51 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672, filePath:TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz -17-08-2023 18:12:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz -17-08-2023 18:12:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz] -17-08-2023 18:12:51 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2269349621436228672/0/0_0_1692321170571.json.gz] -17-08-2023 18:12:51 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0/insertFiles?requestId=af7408a7-5401-4c89-b0c0-fd6f213e25dc&showSkippedFiles=false -17-08-2023 18:12:51 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:12:51 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574301308, X-Snowflake-Connection-Request: 341, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.103, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.410, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.19.45:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:12:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:12:54 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:12:59 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:04 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:09 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:14 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:19 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: cleaner terminated -17-08-2023 18:13:22 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:13:22 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:13:22 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:13:22 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2269349621436228672","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1253,"average_commit_lag_file_count":1,"start_time":1692321169652,"end_time":1692321202791,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:13:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0: service closed -17-08-2023 18:13:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 -17-08-2023 18:13:22 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 is:0, names:[] -17-08-2023 18:13:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2269349621436228672 dropped -17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2269349621436228672_0 dropped -17-08-2023 18:13:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:13:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 -17-08-2023 18:13:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:23 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:13:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9008642977022007506 -17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:13:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 -17-08-2023 18:13:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:24 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:13:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:13:24 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:13:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:13:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, existing:[] -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 - service started -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_9008642977022007506, topicPartition: test-0 -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:13:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:13:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_9008642977022007506. -17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 retrieved 0 file names -17-08-2023 18:13:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506. -17-08-2023 18:13:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 doesn't exist -17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 retrieved 0 file names -17-08-2023 18:13:24 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: cleaner started -17-08-2023 18:13:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_9008642977022007506","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321204019} of type:kafka_pipe_start -17-08-2023 18:13:24 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz -17-08-2023 18:13:24 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 -17-08-2023 18:13:24 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506, filePath:TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz -17-08-2023 18:13:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 -17-08-2023 18:13:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506, filePath:TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz -17-08-2023 18:13:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz -17-08-2023 18:13:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz] -17-08-2023 18:13:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_9008642977022007506/0/1_1_1692321204846.json.gz] -17-08-2023 18:13:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0/insertFiles?requestId=f3fda864-d919-421b-9691-aaf735e677bd&showSkippedFiles=false -17-08-2023 18:13:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:13:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 574947967, X-Snowflake-Connection-Request: 441, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.462, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.20.237:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:13:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:29 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:34 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:39 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:44 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:49 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:54 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: cleaner terminated -17-08-2023 18:13:57 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:13:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:13:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:13:57 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:13:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_9008642977022007506","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1206,"average_commit_lag_file_count":1,"start_time":1692321204018,"end_time":1692321237136,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:13:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0: service closed -17-08-2023 18:13:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 -17-08-2023 18:13:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 is:0, names:[] -17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9008642977022007506 dropped -17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9008642977022007506_0 dropped -17-08-2023 18:13:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:13:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:13:57 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 -17-08-2023 18:13:57 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_603667045474338019 -17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:13:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 -17-08-2023 18:13:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:58 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:13:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:13:58 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:13:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:13:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, existing:[] -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 - service started -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_603667045474338019, topicPartition: test-0 -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_603667045474338019. -17-08-2023 18:13:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 retrieved 0 file names -17-08-2023 18:13:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019. -17-08-2023 18:13:58 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 doesn't exist -17-08-2023 18:13:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:13:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 retrieved 0 file names -17-08-2023 18:13:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_603667045474338019","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321238382} of type:kafka_pipe_start -17-08-2023 18:13:59 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: cleaner started -17-08-2023 18:13:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz -17-08-2023 18:13:59 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -17-08-2023 18:13:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019, filePath:TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz -17-08-2023 18:13:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 -17-08-2023 18:13:59 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019, filePath:TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz -17-08-2023 18:13:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz -17-08-2023 18:13:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz] -17-08-2023 18:13:59 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_603667045474338019/0/0_0_1692321239220.json.gz] -17-08-2023 18:13:59 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0/insertFiles?requestId=a9eeb257-29c2-415b-8fcb-37a8ea09a6c9&showSkippedFiles=false -17-08-2023 18:14:00 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:14:00 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575854182, X-Snowflake-Connection-Request: 274, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.52, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.370, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.21.27:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:14:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:03 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:08 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:13 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:18 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:23 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:28 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: cleaner terminated -17-08-2023 18:14:31 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:14:31 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:14:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:14:31 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_603667045474338019","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1215,"average_commit_lag_file_count":1,"start_time":1692321238381,"end_time":1692321271254,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:14:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0: service closed -17-08-2023 18:14:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 -17-08-2023 18:14:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 is:0, names:[] -17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_603667045474338019 dropped -17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_603667045474338019_0 dropped -17-08-2023 18:14:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 -17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6269835192143780681 -17-08-2023 18:14:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 -17-08-2023 18:14:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:32 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:14:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:14:32 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:14:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -17-08-2023 18:14:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, existing:[] -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 - service started -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6269835192143780681, topicPartition: test-0 -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:14:32 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:14:32 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6269835192143780681. -17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 retrieved 0 file names -17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681. -17-08-2023 18:14:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 doesn't exist -17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:14:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 retrieved 0 file names -17-08-2023 18:14:33 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: cleaner started -17-08-2023 18:14:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6269835192143780681","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692321272495} of type:kafka_pipe_start -17-08-2023 18:14:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz -17-08-2023 18:14:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -17-08-2023 18:14:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681, filePath:TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz -17-08-2023 18:14:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 -17-08-2023 18:14:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681, filePath:TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz -17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz -17-08-2023 18:14:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz] -17-08-2023 18:14:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6269835192143780681/0/1_1_1692321273408.json.gz] -17-08-2023 18:14:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0/insertFiles?requestId=52c59c2f-4302-48b0-acb3-18298072b2ef&showSkippedFiles=false -17-08-2023 18:14:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 01:14:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-Snowflake-Cluster: 17180014582_3653, X-Snowflake-Connection-Id: 575126666, X-Snowflake-Connection-Request: 40, X-Snowflake-Deployment: prod1, X-Snowflake-Nginx: 10.181.190.141, X-Snowflake-Nginx-Version: 1.19.5, X-Snowflake-Redirect-Counter: 0, X-Snowflake-Request-Time: 0.749, X-Snowflake-Status: 200, X-Snowflake-Topology: 1692320814768, X-Snowflake-Upstream: 10.181.23.53:8443, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -17-08-2023 18:14:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:14:37 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:42 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:47 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:52 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:14:57 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:15:02 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:15:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: cleaner terminated -17-08-2023 18:15:05 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -17-08-2023 18:15:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:15:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -17-08-2023 18:15:05 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:15:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6269835192143780681","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1468,"average_commit_lag_file_count":1,"start_time":1692321272493,"end_time":1692321305951,"is_pipe_closing":true} of type:kafka_pipe_usage -17-08-2023 18:15:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0: service closed -17-08-2023 18:15:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 -17-08-2023 18:15:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 is:0, names:[] -17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6269835192143780681 dropped -17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6269835192143780681_0 dropped -17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:06 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:15:06 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:15:06 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:15:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:15:06 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:06 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:15:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:15:06 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:15:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:15:06 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:15:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:15:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -17-08-2023 18:15:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:15:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:15:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:15:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:15:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:15:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3026528099495733147 doesn't exist -17-08-2023 18:15:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3026528099495733147. -17-08-2023 18:15:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3026528099495733147 -17-08-2023 18:15:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:15:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:15:08 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:15:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:15:08 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:15:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:15:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:15:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:15:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -17-08-2023 18:15:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:15:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_8031857255777630677 doesn't exist -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_8031857255777630677. -17-08-2023 18:15:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1050 -17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8031857255777630677 -17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:15:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:15:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:15:08 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:15:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:15:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:15:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:15:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 -17-08-2023 18:15:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:15:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:15:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:15:09 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4437478255481855804 doesn't exist -17-08-2023 18:15:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4437478255481855804. -17-08-2023 18:15:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -17-08-2023 18:15:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1049 -17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4437478255481855804 -17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:09 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:15:09 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:15:09 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:15:09 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:09 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:15:09 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:15:09 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:15:09 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:15:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:15:09 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 -17-08-2023 18:15:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:15:09 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:15:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:15:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:15:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1050, currentDiffMillis=1050 -17-08-2023 18:15:10 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6505219357856853769 doesn't exist -17-08-2023 18:15:10 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6505219357856853769. -17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1047 -17-08-2023 18:15:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6505219357856853769 -17-08-2023 18:15:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:15:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 -17-08-2023 18:16:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:16:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:16:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:16:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:16:35 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:16:35 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:16:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:16:35 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:16:35 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:16:35 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -17-08-2023 18:16:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:16:35 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:16:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:16:35 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:16:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:16:35 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -17-08-2023 18:16:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:16:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:16:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:16:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:16:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:16:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1722213013389090901 doesn't exist -17-08-2023 18:16:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1722213013389090901. -17-08-2023 18:16:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1722213013389090901 -17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1722213013389090901 -17-08-2023 18:16:36 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1722213013389090901, client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:16:36 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_0","table":"kafka_connector_test_table_1722213013389090901"} -17-08-2023 18:16:36 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1722213013389090901, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:16:36 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901 -17-08-2023 18:16:36 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:36 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset:null -17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -17-08-2023 18:16:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1722213013389090901, topicPartition: test-0 -17-08-2023 18:16:36 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -17-08-2023 18:16:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -17-08-2023 18:16:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -17-08-2023 18:16:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:16:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1049 -17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 -17-08-2023 18:16:36 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1ade3eba[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec] -17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -17-08-2023 18:16:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, idx=0 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. -17-08-2023 18:16:36 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -17-08-2023 18:16:36 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -17-08-2023 18:16:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1104: footer length = 754 -17-08-2023 18:16:37 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 754 => 242 2 0 0 -17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1112, encryptedCompressedSize=1120, bdecVersion=THREE -17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, size=1120 -17-08-2023 18:16:37 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, size=1120, timeInMillis=460 -17-08-2023 18:16:37 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec, idx=0 -17-08-2023 18:16:37 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 -17-08-2023 18:16:37 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -17-08-2023 18:16:37 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec","md5":"acf9ad2c9a58b0c7e5ebfdf72bf2ebb5","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","chunk_start_offset":0,"chunk_length":1112,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"e18d3c69c900cd5479811d18a15859ff","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370132722,"first_insert_time_in_ms":1692321396716,"last_insert_time_in_ms":1692321396716,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321396880,"build_duration_ms":229,"upload_duration_ms":458},"bdec_version":3}],"role":"testrole_kafka","request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_2"} -17-08-2023 18:16:37 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/16/rzkbjo_YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_19_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -17-08-2023 18:16:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -17-08-2023 18:16:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1050 -17-08-2023 18:16:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 -17-08-2023 18:16:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:16:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -17-08-2023 18:16:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:16:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -17-08-2023 18:16:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 -17-08-2023 18:16:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1052 -17-08-2023 18:16:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:16:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:16:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 -17-08-2023 18:16:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 -17-08-2023 18:16:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -17-08-2023 18:16:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:16:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:16:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -17-08-2023 18:16:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -17-08-2023 18:16:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -17-08-2023 18:16:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -17-08-2023 18:16:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:16:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:16:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1038 -17-08-2023 18:16:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -17-08-2023 18:16:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -17-08-2023 18:16:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:16:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -17-08-2023 18:16:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -17-08-2023 18:17:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -17-08-2023 18:17:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -17-08-2023 18:17:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -17-08-2023 18:17:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:17:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -17-08-2023 18:17:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -17-08-2023 18:17:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0 -17-08-2023 18:17:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, offset:0 -17-08-2023 18:17:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -17-08-2023 18:17:07 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901.TEST_0, channel sequencer=0, row sequencer=1 -17-08-2023 18:17:07 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"YG8ByLaVJL0mTBBt1iXdCyp2vnPsG3hl8bKG8SBBMLxsgCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1722213013389090901","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:08 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -17-08-2023 18:17:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=958, currentDiffMillis=958 -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=311, min=0, max=41, mean=0.46869509192368386, stddev=2.9561123487373013, p50=0.0, p75=0.0, p95=1.0, p98=11.0, p99=16.0, p999=41.0 -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.3980447542096552, rate_unit=events/second -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=34.43323915113383, rate_unit=events/second -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=229.239667, max=229.239667, mean=229.239667, stddev=0.0, p50=229.239667, p75=229.239667, p95=229.239667, p98=229.239667, p99=229.239667, p999=229.239667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074203691484075, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=860.656333, max=860.656333, mean=860.656333, stddev=0.0, p50=860.656333, p75=860.656333, p95=860.656333, p98=860.656333, p99=860.656333, p999=860.656333, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.030741800766580445, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=168.713084, max=168.713084, mean=168.713084, stddev=0.0, p50=168.713084, p75=168.713084, p95=168.713084, p98=168.713084, p99=168.713084, p999=168.713084, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074155725816862, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=458.591583, max=458.591583, mean=458.591583, stddev=0.0, p50=458.591583, p75=458.591583, p95=458.591583, p98=458.591583, p99=458.591583, p999=458.591583, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03074134131822672, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:08 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:17:08 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -17-08-2023 18:17:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:17:08 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -17-08-2023 18:17:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:17:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:08 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:17:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:17:08 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:17:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:17:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:17:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -17-08-2023 18:17:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:17:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:17:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:17:09 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_7960705935937769956 doesn't exist -17-08-2023 18:17:09 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_7960705935937769956. -17-08-2023 18:17:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7960705935937769956 -17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_7960705935937769956 -17-08-2023 18:17:09 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7960705935937769956, client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_0","table":"kafka_connector_test_table_7960705935937769956"} -17-08-2023 18:17:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7960705935937769956, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:09 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956 -17-08-2023 18:17:09 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset:null -17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -17-08-2023 18:17:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7960705935937769956, topicPartition: test-0 -17-08-2023 18:17:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:17:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:17:09 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} -17-08-2023 18:17:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 -17-08-2023 18:17:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -17-08-2023 18:17:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 -17-08-2023 18:17:09 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@627a88de[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -17-08-2023 18:17:09 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, idx=0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1155: footer length = 775 -17-08-2023 18:17:09 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 775 => 7 3 0 0 -17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1163, encryptedCompressedSize=1168, bdecVersion=THREE -17-08-2023 18:17:09 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, size=1168 -17-08-2023 18:17:10 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, size=1168, timeInMillis=359 -17-08-2023 18:17:10 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec, idx=0 -17-08-2023 18:17:10 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 -17-08-2023 18:17:10 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -17-08-2023 18:17:10 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec","md5":"e530356d0ad4914ffc1892c49b67d004","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","chunk_start_offset":0,"chunk_length":1163,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"73d9563412656d766f9c49ce85954ee5","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370136334,"first_insert_time_in_ms":1692321429613,"last_insert_time_in_ms":1692321429613,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321429947,"build_duration_ms":14,"upload_duration_ms":358},"bdec_version":3}],"role":"testrole_kafka","request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_2"} -17-08-2023 18:17:10 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/17/rzkbkl_kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_31_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -17-08-2023 18:17:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -17-08-2023 18:17:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -17-08-2023 18:17:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -17-08-2023 18:17:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -17-08-2023 18:17:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:17:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:17:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:17:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -17-08-2023 18:17:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -17-08-2023 18:17:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -17-08-2023 18:17:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -17-08-2023 18:17:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:17:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -17-08-2023 18:17:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -17-08-2023 18:17:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -17-08-2023 18:17:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -17-08-2023 18:17:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -17-08-2023 18:17:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:17:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1030 -17-08-2023 18:17:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -17-08-2023 18:17:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -17-08-2023 18:17:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -17-08-2023 18:17:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:33 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -17-08-2023 18:17:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -17-08-2023 18:17:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -17-08-2023 18:17:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -17-08-2023 18:17:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -17-08-2023 18:17:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:38 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -17-08-2023 18:17:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -17-08-2023 18:17:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0 -17-08-2023 18:17:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, offset:1 -17-08-2023 18:17:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956.TEST_0, channel sequencer=0, row sequencer=1 -17-08-2023 18:17:40 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"kzEj9A2sEnS6oxciJilVfjpK9pAAYsbL6JQFV6jkIAA1cCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7960705935937769956","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:40 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -17-08-2023 18:17:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -17-08-2023 18:17:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=806, currentDiffMillis=806 -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=10, mean=0.14629850214578627, stddev=0.8703954391484999, p50=0.0, p75=0.0, p95=1.0, p98=2.0, p99=6.0, p999=10.0 -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.588906402148592, rate_unit=events/second -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.43167012569987, rate_unit=events/second -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=14.945459, max=14.945459, mean=14.945459, stddev=0.0, p50=14.945459, p75=14.945459, p95=14.945459, p98=14.945459, p99=14.945459, p999=14.945459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119125744053028, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=560.997667, max=560.997667, mean=560.997667, stddev=0.0, p50=560.997667, p75=560.997667, p95=560.997667, p98=560.997667, p99=560.997667, p999=560.997667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119105155210925, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=184.702625, max=184.702625, mean=184.702625, stddev=0.0, p50=184.702625, p75=184.702625, p95=184.702625, p98=184.702625, p99=184.702625, p999=184.702625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031190884338964015, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:40 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=358.91875, max=358.91875, mean=358.91875, stddev=0.0, p50=358.91875, p75=358.91875, p95=358.91875, p98=358.91875, p99=358.91875, p999=358.91875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03119069090125127, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:17:40 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:17:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -17-08-2023 18:17:40 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:41 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:17:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:17:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:17:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:41 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:17:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:17:41 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:17:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:17:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:17:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:17:41 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 -17-08-2023 18:17:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:17:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:17:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:17:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:17:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:17:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4166363975679526244 doesn't exist -17-08-2023 18:17:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4166363975679526244. -17-08-2023 18:17:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4166363975679526244 -17-08-2023 18:17:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4166363975679526244 -17-08-2023 18:17:41 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4166363975679526244, client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:17:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_0","table":"kafka_connector_test_table_4166363975679526244"} -17-08-2023 18:17:42 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4166363975679526244, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:17:42 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244 -17-08-2023 18:17:42 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:42 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset:null -17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -17-08-2023 18:17:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4166363975679526244, topicPartition: test-0 -17-08-2023 18:17:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -17-08-2023 18:17:42 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -17-08-2023 18:17:42 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -17-08-2023 18:17:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:17:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 -17-08-2023 18:17:42 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@655305a[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, idx=0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1104: footer length = 754 -17-08-2023 18:17:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 754 => 242 2 0 0 -17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1112, encryptedCompressedSize=1120, bdecVersion=THREE -17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, size=1120 -17-08-2023 18:17:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, size=1120, timeInMillis=241 -17-08-2023 18:17:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec, idx=0 -17-08-2023 18:17:42 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 -17-08-2023 18:17:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -17-08-2023 18:17:42 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec","md5":"917a0ad4ef72db2517f75b653c7ee6a7","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","chunk_start_offset":0,"chunk_length":1112,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"b11b3389c80bd9ed979cc8292da58dd7","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370137822,"first_insert_time_in_ms":1692321462313,"last_insert_time_in_ms":1692321462313,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321462644,"build_duration_ms":11,"upload_duration_ms":241},"bdec_version":3}],"role":"testrole_kafka","request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_2"} -17-08-2023 18:17:43 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/17/rzkbli_43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_39_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -17-08-2023 18:17:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -17-08-2023 18:17:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1020, currentDiffMillis=1021 -17-08-2023 18:17:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1015, currentDiffMillis=1015 -17-08-2023 18:17:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:46 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:17:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -17-08-2023 18:17:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -17-08-2023 18:17:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:17:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:17:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:51 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -17-08-2023 18:17:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -17-08-2023 18:17:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -17-08-2023 18:17:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -17-08-2023 18:17:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -17-08-2023 18:17:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:56 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:17:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:17:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -17-08-2023 18:17:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:17:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -17-08-2023 18:17:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -17-08-2023 18:18:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -17-08-2023 18:18:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:01 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1047 -17-08-2023 18:18:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 -17-08-2023 18:18:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:18:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -17-08-2023 18:18:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:18:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:06 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -17-08-2023 18:18:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -17-08-2023 18:18:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -17-08-2023 18:18:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -17-08-2023 18:18:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:11 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -17-08-2023 18:18:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -17-08-2023 18:18:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0 -17-08-2023 18:18:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:18:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, offset:0 -17-08-2023 18:18:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244.TEST_0, channel sequencer=0, row sequencer=1 -17-08-2023 18:18:13 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"43jMGoeY4FK7xj5B2E8b4BBSFNTFgfR5BFr8FBB2PJ7N4CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4166363975679526244","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:18:13 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -17-08-2023 18:18:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -17-08-2023 18:18:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=757, currentDiffMillis=758 -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=6, mean=0.07622377515040273, stddev=0.46354309399795246, p50=0.0, p75=0.0, p95=0.0, p98=2.0, p99=2.0, p999=6.0 -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.439352093933107, rate_unit=events/second -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.026384399007306, rate_unit=events/second -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=11.616833, max=11.616833, mean=11.616833, stddev=0.0, p50=11.616833, p75=11.616833, p95=11.616833, p98=11.616833, p99=11.616833, p999=11.616833, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127327004150369, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=435.93925, max=435.93925, mean=435.93925, stddev=0.0, p50=435.93925, p75=435.93925, p95=435.93925, p98=435.93925, p99=435.93925, p999=435.93925, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127301771504547, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=181.473708, max=181.473708, mean=181.473708, stddev=0.0, p50=181.473708, p75=181.473708, p95=181.473708, p98=181.473708, p99=181.473708, p999=181.473708, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127282549782918, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:13 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=241.212625, max=241.212625, mean=241.212625, stddev=0.0, p50=241.212625, p75=241.212625, p95=241.212625, p98=241.212625, p99=241.212625, p999=241.212625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031272494777512684, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:13 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:18:13 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:18:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -17-08-2023 18:18:13 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:13 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -17-08-2023 18:18:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -17-08-2023 18:18:14 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -17-08-2023 18:18:14 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -17-08-2023 18:18:14 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:14 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -17-08-2023 18:18:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -17-08-2023 18:18:14 main INFO SecurityManager:190 - Successfully created new JWT -17-08-2023 18:18:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -17-08-2023 18:18:14 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -17-08-2023 18:18:14 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 -17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -17-08-2023 18:18:14 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -17-08-2023 18:18:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6835116767498869672 doesn't exist -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6835116767498869672. -17-08-2023 18:18:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6835116767498869672 -17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_6835116767498869672 -17-08-2023 18:18:14 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6835116767498869672, client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_0","table":"kafka_connector_test_table_6835116767498869672"} -17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6835116767498869672, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:14 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672 -17-08-2023 18:18:14 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:14 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset:null -17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6835116767498869672, topicPartition: test-0 -17-08-2023 18:18:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -17-08-2023 18:18:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -17-08-2023 18:18:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} -17-08-2023 18:18:14 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 -17-08-2023 18:18:14 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -17-08-2023 18:18:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 -17-08-2023 18:18:15 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@37654ab6[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, enableParquetMemoryOptimization=false -17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, idx=0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1112: footer length = 750 -17-08-2023 18:18:15 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 750 => 238 2 0 0 -17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1120, encryptedCompressedSize=1136, bdecVersion=THREE -17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, size=1136 -17-08-2023 18:18:15 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, size=1136, timeInMillis=249 -17-08-2023 18:18:15 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec, idx=0 -17-08-2023 18:18:15 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 -17-08-2023 18:18:15 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -17-08-2023 18:18:15 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec","md5":"bc9ff0cc0a667cd1922c134e1dc49a48","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","chunk_start_offset":0,"chunk_length":1120,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"9710545448fc26ceb55354751239a1a1","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370141602,"first_insert_time_in_ms":1692321494973,"last_insert_time_in_ms":1692321494973,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692321495254,"build_duration_ms":11,"upload_duration_ms":248},"bdec_version":3}],"role":"testrole_kafka","request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_2"} -17-08-2023 18:18:15 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/1/18/rzkbmf_9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_47_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -17-08-2023 18:18:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1025 -17-08-2023 18:18:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -17-08-2023 18:18:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -17-08-2023 18:18:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -17-08-2023 18:18:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -17-08-2023 18:18:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -17-08-2023 18:18:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -17-08-2023 18:18:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1026 -17-08-2023 18:18:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -17-08-2023 18:18:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -17-08-2023 18:18:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -17-08-2023 18:18:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -17-08-2023 18:18:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:18:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -17-08-2023 18:18:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -17-08-2023 18:18:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -17-08-2023 18:18:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -17-08-2023 18:18:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -17-08-2023 18:18:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -17-08-2023 18:18:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -17-08-2023 18:18:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -17-08-2023 18:18:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -17-08-2023 18:18:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:18:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -17-08-2023 18:18:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 -17-08-2023 18:18:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -17-08-2023 18:18:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1024 -17-08-2023 18:18:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -17-08-2023 18:18:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -17-08-2023 18:18:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -17-08-2023 18:18:46 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0 -17-08-2023 18:18:46 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:18:46 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, offset:1 -17-08-2023 18:18:46 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672.TEST_0, channel sequencer=0, row sequencer=1 -17-08-2023 18:18:46 ForkJoinPool.commonPool-worker-5 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"9y0MqAAMViClip09hLfqff25HD2vMPn7VaDnKpM4CCdUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6835116767498869672","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -17-08-2023 18:18:46 ForkJoinPool.commonPool-worker-5 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -17-08-2023 18:18:46 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -17-08-2023 18:18:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=942, currentDiffMillis=943 -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=311, min=0, max=13, mean=0.2378348527545219, stddev=1.2813385457487345, p50=0.0, p75=0.0, p95=1.0, p98=5.0, p99=8.0, p999=13.0 -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.3896226230756414, rate_unit=events/second -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.25453620654256, rate_unit=events/second -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=11.89775, max=11.89775, mean=11.89775, stddev=0.0, p50=11.89775, p75=11.89775, p95=11.89775, p98=11.89775, p99=11.89775, p999=11.89775, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033665007850115, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=431.422375, max=431.422375, mean=431.422375, stddev=0.0, p50=431.422375, p75=431.422375, p95=431.422375, p98=431.422375, p99=431.422375, p999=431.422375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033466453044868, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=169.58025, max=169.58025, mean=169.58025, stddev=0.0, p50=169.58025, p75=169.58025, p95=169.58025, p98=169.58025, p99=169.58025, p999=169.58025, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03103331677537786, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:46 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=248.38375, max=248.38375, mean=248.38375, stddev=0.0, p50=248.38375, p75=248.38375, p95=248.38375, p98=248.38375, p99=248.38375, p999=248.38375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031033147356489293, rate_unit=events/second, duration_unit=milliseconds -17-08-2023 18:18:46 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:46 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -17-08-2023 18:18:46 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -17-08-2023 18:18:46 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -17-08-2023 18:18:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:39:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:39:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 -18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4574998329344794831 -18-08-2023 12:39:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 -18-08-2023 12:39:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:39:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 12:39:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:39:32 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:39:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:39:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, existing:[] -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 - service started -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4574998329344794831, topicPartition: test-0 -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4574998329344794831. -18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 retrieved 0 file names -18-08-2023 12:39:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831. -18-08-2023 12:39:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 doesn't exist -18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:39:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 retrieved 0 file names -18-08-2023 12:39:33 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: cleaner started -18-08-2023 12:39:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4574998329344794831","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387572739} of type:kafka_pipe_start -18-08-2023 12:39:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz -18-08-2023 12:39:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 12:39:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831, filePath:TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz -18-08-2023 12:39:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 -18-08-2023 12:39:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831, filePath:TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz -18-08-2023 12:39:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz -18-08-2023 12:39:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz] -18-08-2023 12:39:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4574998329344794831/0/0_0_1692387573562.json.gz] -18-08-2023 12:39:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0/insertFiles?requestId=bf5cd097-b311-42ab-b339-6bb58f641558&showSkippedFiles=false -18-08-2023 12:39:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:39:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:39:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:39:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:39:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:39:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:39:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:39:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: cleaner terminated -18-08-2023 12:40:05 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:40:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:40:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:40:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4574998329344794831","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1291,"average_commit_lag_file_count":1,"start_time":1692387572727,"end_time":1692387605673,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:40:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0: service closed -18-08-2023 12:40:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 -18-08-2023 12:40:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 is:0, names:[] -18-08-2023 12:40:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4574998329344794831 dropped -18-08-2023 12:40:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4574998329344794831_0 dropped -18-08-2023 12:40:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 -18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3789295266337469155 -18-08-2023 12:40:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 -18-08-2023 12:40:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:06 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:40:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:40:06 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:40:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:40:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, existing:[] -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 - service started -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3789295266337469155, topicPartition: test-0 -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:40:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 12:40:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3789295266337469155. -18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 retrieved 0 file names -18-08-2023 12:40:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155. -18-08-2023 12:40:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 doesn't exist -18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 retrieved 0 file names -18-08-2023 12:40:07 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: cleaner started -18-08-2023 12:40:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3789295266337469155","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387606758} of type:kafka_pipe_start -18-08-2023 12:40:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz -18-08-2023 12:40:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 12:40:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155, filePath:TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz -18-08-2023 12:40:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 -18-08-2023 12:40:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155, filePath:TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz -18-08-2023 12:40:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz -18-08-2023 12:40:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz] -18-08-2023 12:40:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3789295266337469155/0/1_1_1692387607567.json.gz] -18-08-2023 12:40:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0/insertFiles?requestId=3797ef7f-5ebf-4165-bb3f-ff0a94265c55&showSkippedFiles=false -18-08-2023 12:40:08 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:40:08 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:40:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:26 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:36 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: cleaner terminated -18-08-2023 12:40:39 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:40:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:40:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:40:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3789295266337469155","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1163,"average_commit_lag_file_count":1,"start_time":1692387606757,"end_time":1692387639612,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:40:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0: service closed -18-08-2023 12:40:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 -18-08-2023 12:40:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 is:0, names:[] -18-08-2023 12:40:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3789295266337469155 dropped -18-08-2023 12:40:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3789295266337469155_0 dropped -18-08-2023 12:40:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 -18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7746804880075842119 -18-08-2023 12:40:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 -18-08-2023 12:40:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:40 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:40:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:40:40 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:40:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:40:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:40 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, existing:[] -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 - service started -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7746804880075842119, topicPartition: test-0 -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7746804880075842119. -18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 retrieved 0 file names -18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119. -18-08-2023 12:40:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 doesn't exist -18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:40:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 retrieved 0 file names -18-08-2023 12:40:41 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: cleaner started -18-08-2023 12:40:41 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7746804880075842119","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387640571} of type:kafka_pipe_start -18-08-2023 12:40:41 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz -18-08-2023 12:40:41 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 12:40:41 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119, filePath:TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz -18-08-2023 12:40:41 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 -18-08-2023 12:40:41 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119, filePath:TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz -18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz -18-08-2023 12:40:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz] -18-08-2023 12:40:41 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7746804880075842119/0/0_0_1692387641333.json.gz] -18-08-2023 12:40:41 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0/insertFiles?requestId=d865cd6b-f441-461f-8d12-04745d7a350a&showSkippedFiles=false -18-08-2023 12:40:42 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:40:42 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:40:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:40:45 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:50 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:40:55 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:00 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:05 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:10 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: cleaner terminated -18-08-2023 12:41:13 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:41:13 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:41:13 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:41:13 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:13 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7746804880075842119","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1049,"average_commit_lag_file_count":1,"start_time":1692387640562,"end_time":1692387673089,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:41:13 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0: service closed -18-08-2023 12:41:13 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 -18-08-2023 12:41:13 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 is:0, names:[] -18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7746804880075842119 dropped -18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7746804880075842119_0 dropped -18-08-2023 12:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 -18-08-2023 12:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2936916385938205034 -18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:41:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 -18-08-2023 12:41:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:14 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:41:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:41:14 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:41:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:41:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:14 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, existing:[] -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 - service started -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2936916385938205034, topicPartition: test-0 -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:41:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 12:41:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2936916385938205034. -18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 retrieved 0 file names -18-08-2023 12:41:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034. -18-08-2023 12:41:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 doesn't exist -18-08-2023 12:41:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 retrieved 0 file names -18-08-2023 12:41:15 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: cleaner started -18-08-2023 12:41:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2936916385938205034","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387674196} of type:kafka_pipe_start -18-08-2023 12:41:15 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz -18-08-2023 12:41:15 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 12:41:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034, filePath:TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz -18-08-2023 12:41:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 -18-08-2023 12:41:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034, filePath:TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz -18-08-2023 12:41:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz -18-08-2023 12:41:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz] -18-08-2023 12:41:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2936916385938205034/0/1_1_1692387675050.json.gz] -18-08-2023 12:41:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0/insertFiles?requestId=2f41ab2f-9601-4784-a693-bf327aedfd1c&showSkippedFiles=false -18-08-2023 12:41:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:41:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:34 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:39 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:44 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: cleaner terminated -18-08-2023 12:41:47 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:41:47 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:41:47 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:41:47 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:47 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2936916385938205034","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1015,"average_commit_lag_file_count":1,"start_time":1692387674175,"end_time":1692387707047,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:41:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0: service closed -18-08-2023 12:41:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 -18-08-2023 12:41:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 is:0, names:[] -18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2936916385938205034 dropped -18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2936916385938205034_0 dropped -18-08-2023 12:41:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:41:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 -18-08-2023 12:41:47 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:47 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 12:41:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4529001433918421359 -18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:41:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 -18-08-2023 12:41:48 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:48 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:41:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:41:48 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:41:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:41:48 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:48 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, existing:[] -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 - service started -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4529001433918421359, topicPartition: test-0 -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4529001433918421359. -18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 retrieved 0 file names -18-08-2023 12:41:48 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359. -18-08-2023 12:41:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 doesn't exist -18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:41:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 retrieved 0 file names -18-08-2023 12:41:48 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: cleaner started -18-08-2023 12:41:48 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4529001433918421359","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387708039} of type:kafka_pipe_start -18-08-2023 12:41:48 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz -18-08-2023 12:41:48 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 12:41:48 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359, filePath:TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz -18-08-2023 12:41:49 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 -18-08-2023 12:41:49 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359, filePath:TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz -18-08-2023 12:41:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz -18-08-2023 12:41:49 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz] -18-08-2023 12:41:49 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4529001433918421359/0/0_0_1692387708823.json.gz] -18-08-2023 12:41:49 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0/insertFiles?requestId=683d72b3-2346-47f0-a506-995cc115d6b1&showSkippedFiles=false -18-08-2023 12:41:49 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:49 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:41:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:53 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_1 -18-08-2023 12:41:56 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:41:56 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:41:56 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_191448210870743606 -18-08-2023 12:41:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:41:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:41:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 -18-08-2023 12:41:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 -18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:41:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:56 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:41:57 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 12:41:57 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:41:57 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:41:57 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:41:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:57 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, existing:[] -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 - service started -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_191448210870743606, topicPartition: test-0 -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_191448210870743606. -18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names -18-08-2023 12:41:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606. -18-08-2023 12:41:57 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 doesn't exist -18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:41:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names -18-08-2023 12:41:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: cleaner started -18-08-2023 12:41:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387717118} of type:kafka_pipe_start -18-08-2023 12:41:57 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz -18-08-2023 12:41:57 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 3 records, 7136 bytes, offset 0 - 2 -18-08-2023 12:41:58 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:41:58 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606, filePath:TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz -18-08-2023 12:41:58 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:41:58 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606, filePath:TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz -18-08-2023 12:41:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz -18-08-2023 12:41:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names -18-08-2023 12:41:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] -18-08-2023 12:41:58 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] -18-08-2023 12:41:58 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0/insertFiles?requestId=5e0180ed-bc83-4a7a-80c4-965e4fe8a9aa&showSkippedFiles=false -18-08-2023 12:41:59 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:41:59 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:41:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names -18-08-2023 12:41:59 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:03 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:08 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:13 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:17 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:18 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:20 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:42:20 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: cleaner terminated -18-08-2023 12:42:20 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:42:20 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:42:20 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:20 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4529001433918421359","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":900,"average_commit_lag_file_count":1,"start_time":1692387708038,"end_time":1692387740450,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:42:20 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0: service closed -18-08-2023 12:42:20 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 -18-08-2023 12:42:20 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 is:0, names:[] -18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4529001433918421359 dropped -18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4529001433918421359_0 dropped -18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 -18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:21 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8187446402040964935 -18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 -18-08-2023 12:42:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:21 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:42:21 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:42:21 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:42:21 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:42:21 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:21 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, existing:[] -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 - service started -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8187446402040964935, topicPartition: test-0 -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:42:21 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 12:42:21 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8187446402040964935. -18-08-2023 12:42:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 retrieved 0 file names -18-08-2023 12:42:21 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935. -18-08-2023 12:42:21 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 doesn't exist -18-08-2023 12:42:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 retrieved 0 file names -18-08-2023 12:42:22 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: cleaner started -18-08-2023 12:42:22 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8187446402040964935","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387741467} of type:kafka_pipe_start -18-08-2023 12:42:22 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz -18-08-2023 12:42:22 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 -18-08-2023 12:42:22 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935, filePath:TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz -18-08-2023 12:42:22 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 -18-08-2023 12:42:22 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935, filePath:TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz -18-08-2023 12:42:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz -18-08-2023 12:42:22 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz] -18-08-2023 12:42:22 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8187446402040964935/0/1_1_1692387742154.json.gz] -18-08-2023 12:42:22 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0/insertFiles?requestId=aa386756-be1f-4ffe-af99-9577fb3972e6&showSkippedFiles=false -18-08-2023 12:42:23 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:23 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:42:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:26 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 1 file names -18-08-2023 12:42:31 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:36 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:42:41 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:42:46 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:42:51 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:42:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: cleaner terminated -18-08-2023 12:42:54 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:42:54 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:42:54 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:42:54 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:42:54 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8187446402040964935","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1198,"average_commit_lag_file_count":1,"start_time":1692387741466,"end_time":1692387774319,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:42:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0: service closed -18-08-2023 12:42:54 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 -18-08-2023 12:42:54 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 is:0, names:[] -18-08-2023 12:42:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8187446402040964935 dropped -18-08-2023 12:42:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8187446402040964935_0 dropped -18-08-2023 12:42:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 -18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7966792751296594456 -18-08-2023 12:42:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 -18-08-2023 12:42:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:55 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:42:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:42:55 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:42:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:42:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:55 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, existing:[] -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 - service started -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7966792751296594456, topicPartition: test-0 -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7966792751296594456. -18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 retrieved 0 file names -18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456. -18-08-2023 12:42:56 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 doesn't exist -18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:42:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 retrieved 0 file names -18-08-2023 12:42:56 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: cleaner started -18-08-2023 12:42:56 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7966792751296594456","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387775534} of type:kafka_pipe_start -18-08-2023 12:42:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz -18-08-2023 12:42:56 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 12:42:56 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456, filePath:TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz -18-08-2023 12:42:56 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 -18-08-2023 12:42:56 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456, filePath:TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz -18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz -18-08-2023 12:42:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz] -18-08-2023 12:42:56 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7966792751296594456/0/0_0_1692387776340.json.gz] -18-08-2023 12:42:56 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0/insertFiles?requestId=c1ead534-38e3-4ad3-9519-3d9537117027&showSkippedFiles=false -18-08-2023 12:42:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:42:57 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:57 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:42:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:42:57 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0/insertReport?requestId=3644d08b-51db-4538-b30e-9a19fb61fe0d -18-08-2023 12:42:58 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:42:58 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@53c25f1b} -18-08-2023 12:42:58 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 1 -18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] Purging loaded files for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0, loadedFileCount:1, loadedFiles:[TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz] -18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] deleted TEST_CONNECTOR/kafka_connector_test_table_191448210870743606/0/0_2_1692387717827.json.gz from stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:42:58 pool-4-thread-1 INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] purge 1 files from stage: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 -18-08-2023 12:42:58 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":7136,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":59989,"average_ingestion_lag_file_count":1,"average_commit_lag":1381,"average_commit_lag_file_count":1,"start_time":1692387717114,"end_time":1692387778353,"is_pipe_closing":false} of type:kafka_pipe_usage -18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 retrieved 0 file names -18-08-2023 12:43:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: cleaner terminated -18-08-2023 12:43:00 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:43:00 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:43:00 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:43:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_191448210870743606","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":7136,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692387778353,"end_time":1692387780165,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:43:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0: service closed -18-08-2023 12:43:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 -18-08-2023 12:43:00 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 is:0, names:[] -18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_191448210870743606 dropped -18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_0 dropped -18-08-2023 12:43:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_191448210870743606_1 dropped -18-08-2023 12:43:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:00 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:05 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:10 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:15 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 -18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_1 -18-08-2023 12:43:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:43:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:43:16 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8464995679227078074 -18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 -18-08-2023 12:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 -18-08-2023 12:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:17 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:43:17 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 12:43:17 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:43:17 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:43:17 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:43:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:17 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0, existing:[] -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 - service started -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8464995679227078074, topicPartition: test-0 -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8464995679227078074. -18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 retrieved 0 file names -18-08-2023 12:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074. -18-08-2023 12:43:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 doesn't exist -18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 -18-08-2023 12:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 retrieved 0 file names -18-08-2023 12:43:17 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0: cleaner started -18-08-2023 12:43:18 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8464995679227078074","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387797337} of type:kafka_pipe_start -18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8464995679227078074 dropped -18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_0 dropped -18-08-2023 12:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8464995679227078074_1 dropped -18-08-2023 12:43:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:20 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:25 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:28 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: cleaner terminated -18-08-2023 12:43:28 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:43:28 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:43:28 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:43:28 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:28 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7966792751296594456","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1124,"average_commit_lag_file_count":1,"start_time":1692387775533,"end_time":1692387808167,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:43:28 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0: service closed -18-08-2023 12:43:28 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 -18-08-2023 12:43:28 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 is:0, names:[] -18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7966792751296594456 dropped -18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7966792751296594456_0 dropped -18-08-2023 12:43:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:43:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 -18-08-2023 12:43:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8912050872937575474 -18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:43:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 -18-08-2023 12:43:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:29 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:43:29 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:43:29 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:43:29 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:43:29 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:29 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, existing:[] -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 - service started -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8912050872937575474, topicPartition: test-0 -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:43:29 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 12:43:29 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8912050872937575474. -18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 retrieved 0 file names -18-08-2023 12:43:29 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474. -18-08-2023 12:43:29 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 doesn't exist -18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:43:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 retrieved 0 file names -18-08-2023 12:43:29 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: cleaner started -18-08-2023 12:43:29 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8912050872937575474","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387809031} of type:kafka_pipe_start -18-08-2023 12:43:29 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz -18-08-2023 12:43:29 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 12:43:30 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474, filePath:TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz -18-08-2023 12:43:30 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 -18-08-2023 12:43:30 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474, filePath:TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz -18-08-2023 12:43:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz -18-08-2023 12:43:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz] -18-08-2023 12:43:30 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8912050872937575474/0/1_1_1692387809933.json.gz] -18-08-2023 12:43:30 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0/insertFiles?requestId=95cafb91-27c8-4758-918b-b7aaf0e3fc5e&showSkippedFiles=false -18-08-2023 12:43:31 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:43:31 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:43:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:43:34 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:39 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:49 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:54 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:43:59 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: cleaner terminated -18-08-2023 12:44:01 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:44:01 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:44:01 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:44:01 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:01 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8912050872937575474","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1112,"average_commit_lag_file_count":1,"start_time":1692387809029,"end_time":1692387841831,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:44:01 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0: service closed -18-08-2023 12:44:01 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 -18-08-2023 12:44:01 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 is:0, names:[] -18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8912050872937575474 dropped -18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8912050872937575474_0 dropped -18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:02 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:44:02 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 12:44:02 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 12:44:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 12:44:02 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:02 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:44:02 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:44:02 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:44:02 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 12:44:02 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 12:44:02 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -18-08-2023 12:44:02 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 12:44:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 12:44:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 12:44:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:44:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1196559338745700642 doesn't exist -18-08-2023 12:44:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1196559338745700642. -18-08-2023 12:44:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1196559338745700642 -18-08-2023 12:44:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1196559338745700642 -18-08-2023 12:44:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1196559338745700642, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_0","table":"kafka_connector_test_table_1196559338745700642"} -18-08-2023 12:44:03 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1196559338745700642, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:03 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642 -18-08-2023 12:44:03 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:03 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset:null -18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 12:44:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1196559338745700642, topicPartition: test-0 -18-08-2023 12:44:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -18-08-2023 12:44:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -18-08-2023 12:44:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 12:44:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1054, currentDiffMillis=1061 -18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 -18-08-2023 12:44:03 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1777552b[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec] -18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 12:44:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, idx=0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. -18-08-2023 12:44:03 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -18-08-2023 12:44:03 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -18-08-2023 12:44:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -18-08-2023 12:44:04 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1105: footer length = 755 -18-08-2023 12:44:04 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 755 => 243 2 0 0 -18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1113, encryptedCompressedSize=1120, bdecVersion=THREE -18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, size=1120 -18-08-2023 12:44:04 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, size=1120, timeInMillis=207 -18-08-2023 12:44:04 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec, idx=0 -18-08-2023 12:44:04 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 12:44:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 12:44:04 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec","md5":"e623cdb8f87165f05bfc3c1676775646","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","chunk_start_offset":0,"chunk_length":1113,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"0e01e2385f97de443ef3fea72f1f1da0","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370747854,"first_insert_time_in_ms":1692387843349,"last_insert_time_in_ms":1692387843349,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387843805,"build_duration_ms":254,"upload_duration_ms":204},"bdec_version":3}],"role":"testrole_kafka","request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_2"} -18-08-2023 12:44:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/44/rzlqtf_wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_62_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 12:44:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:44:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 12:44:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 12:44:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:07 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 12:44:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 12:44:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:44:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 -18-08-2023 12:44:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 12:44:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:12 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1038 -18-08-2023 12:44:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 12:44:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -18-08-2023 12:44:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 -18-08-2023 12:44:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1046 -18-08-2023 12:44:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:17 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -18-08-2023 12:44:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 -18-08-2023 12:44:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 -18-08-2023 12:44:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:44:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 12:44:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:22 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:44:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:44:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 12:44:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 12:44:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:27 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 12:44:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:44:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:44:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:44:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 12:44:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:32 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:44:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:44:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0 -18-08-2023 12:44:34 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:44:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, offset:0 -18-08-2023 12:44:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 12:44:34 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wP9ea6gAA0NBBnQZjhKned3aUQdIwtBBdNVOIPaiKmoHsUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1196559338745700642","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:44:34 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -18-08-2023 12:44:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 12:44:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=621, currentDiffMillis=621 -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=308, min=0, max=21, mean=0.2548306063738825, stddev=1.7579762641061176, p50=0.0, p75=0.0, p95=0.0, p98=4.0, p99=12.0, p999=21.0 -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4401573727090455, rate_unit=events/second -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.03800807271313, rate_unit=events/second -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=254.494, max=254.494, mean=254.494, stddev=0.0, p50=254.494, p75=254.494, p95=254.494, p98=254.494, p99=254.494, p999=254.494, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03128272466275268, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=621.549875, max=621.549875, mean=621.549875, stddev=0.0, p50=621.549875, p75=621.549875, p95=621.549875, p98=621.549875, p99=621.549875, p999=621.549875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282568575417946, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=157.005708, max=157.005708, mean=157.005708, stddev=0.0, p50=157.005708, p75=157.005708, p95=157.005708, p98=157.005708, p99=157.005708, p999=157.005708, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282468106885765, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:44:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=204.639709, max=204.639709, mean=204.639709, stddev=0.0, p50=204.639709, p75=204.639709, p95=204.639709, p98=204.639709, p99=204.639709, p999=204.639709, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031282337913482866, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:44:34 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:34 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:44:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 12:44:34 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:44:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:44:35 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 12:44:35 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 12:44:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 12:44:35 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:35 Thread-18 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:44:35 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:44:35 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:44:35 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 12:44:35 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 12:44:35 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 12:44:35 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:44:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_9105461817253957002 doesn't exist -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_9105461817253957002. -18-08-2023 12:44:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9105461817253957002 -18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_9105461817253957002 -18-08-2023 12:44:35 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_9105461817253957002, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_0","table":"kafka_connector_test_table_9105461817253957002"} -18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_9105461817253957002, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:44:35 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002 -18-08-2023 12:44:35 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset:null -18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_9105461817253957002, topicPartition: test-0 -18-08-2023 12:44:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:44:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 12:44:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} -18-08-2023 12:44:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 -18-08-2023 12:44:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 12:44:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:44:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 -18-08-2023 12:44:36 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@4e0ddc23[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, idx=0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1155: footer length = 775 -18-08-2023 12:44:36 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 775 => 7 3 0 0 -18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1163, encryptedCompressedSize=1168, bdecVersion=THREE -18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, size=1168 -18-08-2023 12:44:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, size=1168, timeInMillis=287 -18-08-2023 12:44:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec, idx=0 -18-08-2023 12:44:36 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 12:44:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 12:44:36 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec","md5":"126aae5f4fb8c18cdb76f15cfec1887a","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","chunk_start_offset":0,"chunk_length":1163,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"2ac552b25f8368a0281516c1da166a7e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370745286,"first_insert_time_in_ms":1692387875754,"last_insert_time_in_ms":1692387875754,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387876171,"build_duration_ms":9,"upload_duration_ms":286},"bdec_version":3}],"role":"testrole_kafka","request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_2"} -18-08-2023 12:44:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/44/rzlquc_dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_73_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 12:44:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -18-08-2023 12:44:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 -18-08-2023 12:44:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 12:44:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:40 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1022, currentDiffMillis=1023 -18-08-2023 12:44:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:44:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:44:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 12:44:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 12:44:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:45 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 -18-08-2023 12:44:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 12:44:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1049 -18-08-2023 12:44:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1052, currentDiffMillis=1053 -18-08-2023 12:44:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 12:44:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:50 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:44:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 -18-08-2023 12:44:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:44:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -18-08-2023 12:44:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 12:44:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:55 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:44:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -18-08-2023 12:44:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:44:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 12:44:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:44:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 12:44:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:45:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:00 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 12:45:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 12:45:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:45:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 12:45:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:05 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:45:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 12:45:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:06 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0 -18-08-2023 12:45:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:06 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, offset:1 -18-08-2023 12:45:06 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 12:45:06 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"dkMEIFv2R7sl7PBBM0jlAA4SxzfUPkJ8WtkJNSSrfCvJUCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_9105461817253957002","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:06 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 12:45:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 12:45:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=493, currentDiffMillis=493 -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=9, mean=0.09705703854644274, stddev=0.7988878636542661, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=4.0, p999=9.0 -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.6156135321258924, rate_unit=events/second -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.80760150999839, rate_unit=events/second -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=9.518958, max=9.518958, mean=9.518958, stddev=0.0, p50=9.518958, p75=9.518958, p95=9.518958, p98=9.518958, p99=9.518958, p999=9.518958, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0315132618714113, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=434.757167, max=434.757167, mean=434.757167, stddev=0.0, p50=434.757167, p75=434.757167, p95=434.757167, p98=434.757167, p99=434.757167, p999=434.757167, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03151318184579166, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=137.858209, max=137.858209, mean=137.858209, stddev=0.0, p50=137.858209, p75=137.858209, p95=137.858209, p98=137.858209, p99=137.858209, p999=137.858209, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031513128674358286, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=286.275125, max=286.275125, mean=286.275125, stddev=0.0, p50=286.275125, p75=286.275125, p95=286.275125, p98=286.275125, p99=286.275125, p999=286.275125, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03151304310509383, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:06 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:45:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:45:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 12:45:06 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:45:07 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 12:45:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:07 Thread-20 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:45:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:45:07 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:45:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 12:45:07 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 12:45:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 -18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 12:45:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:45:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_7874228197508598970 doesn't exist -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_7874228197508598970. -18-08-2023 12:45:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7874228197508598970 -18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_7874228197508598970 -18-08-2023 12:45:07 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7874228197508598970, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_0","table":"kafka_connector_test_table_7874228197508598970"} -18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_7874228197508598970, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:07 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970 -18-08-2023 12:45:07 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset:null -18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7874228197508598970, topicPartition: test-0 -18-08-2023 12:45:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -18-08-2023 12:45:07 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -18-08-2023 12:45:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 12:45:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1024, currentDiffMillis=1024 -18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 -18-08-2023 12:45:08 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@59f2a0f0[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, idx=0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1102: footer length = 752 -18-08-2023 12:45:08 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 752 => 240 2 0 0 -18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1110, encryptedCompressedSize=1120, bdecVersion=THREE -18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, size=1120 -18-08-2023 12:45:08 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, size=1120, timeInMillis=226 -18-08-2023 12:45:08 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec, idx=0 -18-08-2023 12:45:08 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 12:45:08 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -18-08-2023 12:45:08 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec","md5":"112aaf108f3d79012aca70081d3f4e27","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","chunk_start_offset":0,"chunk_length":1110,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"1fcb88c3acf901bbb3afd81475cc720e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370745514,"first_insert_time_in_ms":1692387907912,"last_insert_time_in_ms":1692387907912,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387908331,"build_duration_ms":4,"upload_duration_ms":225},"bdec_version":3}],"role":"testrole_kafka","request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_2"} -18-08-2023 12:45:08 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/45/rzlqv8_UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -18-08-2023 12:45:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:45:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_1 -18-08-2023 12:45:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:45:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 12:45:10 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 12:45:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:45:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8383920415137154704 -18-08-2023 12:45:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 3 -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 3 -18-08-2023 12:45:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:45:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:11 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:45:11 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 12:45:11 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:45:11 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:45:11 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:45:11 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:11 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, existing:[] -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 - service started -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8383920415137154704, topicPartition: test-0 -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -18-08-2023 12:45:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8383920415137154704. -18-08-2023 12:45:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names -18-08-2023 12:45:11 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704. -18-08-2023 12:45:11 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 doesn't exist -18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names -18-08-2023 12:45:12 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: cleaner started -18-08-2023 12:45:12 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692387911341} of type:kafka_pipe_start -18-08-2023 12:45:12 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz -18-08-2023 12:45:12 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 3 records, 5492 bytes, offset 0 - 2 -18-08-2023 12:45:12 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:12 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704, filePath:TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz -18-08-2023 12:45:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 12:45:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:12 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:45:12 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704, filePath:TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz -18-08-2023 12:45:12 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz -18-08-2023 12:45:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names -18-08-2023 12:45:12 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] -18-08-2023 12:45:12 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] -18-08-2023 12:45:12 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0/insertFiles?requestId=9dd4f6c2-d041-429e-ad4f-0f1d22828e66&showSkippedFiles=false -18-08-2023 12:45:13 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:45:13 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:45:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 12:45:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names -18-08-2023 12:45:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 12:45:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -18-08-2023 12:45:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -18-08-2023 12:45:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:17 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 12:45:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 12:45:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:45:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 12:45:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:21 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 12:45:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:22 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 12:45:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 12:45:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 12:45:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 12:45:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:26 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1046 -18-08-2023 12:45:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:27 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -18-08-2023 12:45:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 12:45:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -18-08-2023 12:45:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 12:45:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:31 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 12:45:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:32 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 12:45:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 12:45:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 12:45:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:36 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -18-08-2023 12:45:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:37 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1022, currentDiffMillis=1022 -18-08-2023 12:45:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 12:45:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0 -18-08-2023 12:45:38 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, offset:0 -18-08-2023 12:45:38 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 12:45:38 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"UjosW2cyZHcfX9YJS1SUQu8zhAqiAVxH7RnMU4tLlZkCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_7874228197508598970","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:38 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -18-08-2023 12:45:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 12:45:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=367, currentDiffMillis=367 -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=305, min=0, max=2, mean=0.01293826210198239, stddev=0.13413729964389748, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=1.0, p999=2.0 -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4735974066205038, rate_unit=events/second -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.51821928265969, rate_unit=events/second -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=4.477, max=4.477, mean=4.477, stddev=0.0, p50=4.477, p75=4.477, p95=4.477, p98=4.477, p99=4.477, p999=4.477, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031712433596499674, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=370.255416, max=370.255416, mean=370.255416, stddev=0.0, p50=370.255416, p75=370.255416, p95=370.255416, p98=370.255416, p99=370.255416, p999=370.255416, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031712061626553704, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=139.045125, max=139.045125, mean=139.045125, stddev=0.0, p50=139.045125, p75=139.045125, p95=139.045125, p98=139.045125, p99=139.045125, p999=139.045125, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03171158273189406, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:38 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=225.996042, max=225.996042, mean=225.996042, stddev=0.0, p50=225.996042, p75=225.996042, p95=225.996042, p98=225.996042, p99=225.996042, p999=225.996042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031711420784943704, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:45:38 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:38 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:45:38 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 12:45:38 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:45:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 12:45:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 12:45:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:39 Thread-22 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:45:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:45:39 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:45:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 12:45:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 12:45:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 -18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 12:45:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:45:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 12:45:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 12:45:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:45:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4720053319416705063 doesn't exist -18-08-2023 12:45:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4720053319416705063. -18-08-2023 12:45:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4720053319416705063 -18-08-2023 12:45:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4720053319416705063 -18-08-2023 12:45:39 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4720053319416705063, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_0","table":"kafka_connector_test_table_4720053319416705063"} -18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4720053319416705063, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:45:39 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063 -18-08-2023 12:45:39 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 12:45:39 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:45:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset:null -18-08-2023 12:45:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 12:45:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4720053319416705063, topicPartition: test-0 -18-08-2023 12:45:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:45:40 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 12:45:40 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} -18-08-2023 12:45:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 -18-08-2023 12:45:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 12:45:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1032 -18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 -18-08-2023 12:45:40 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@28b462e9[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec] -18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, idx=0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1113: footer length = 751 -18-08-2023 12:45:40 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 751 => 239 2 0 0 -18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1121, encryptedCompressedSize=1136, bdecVersion=THREE -18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, size=1136 -18-08-2023 12:45:40 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, size=1136, timeInMillis=199 -18-08-2023 12:45:40 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec, idx=0 -18-08-2023 12:45:40 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 12:45:40 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -18-08-2023 12:45:40 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec","md5":"2ccbb04334296489837e3cab3e54dc3c","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","chunk_start_offset":0,"chunk_length":1121,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"f5cf44207772ff02ae74e8f7c8a0ee82","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370748686,"first_insert_time_in_ms":1692387940000,"last_insert_time_in_ms":1692387940000,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692387940348,"build_duration_ms":8,"upload_duration_ms":198},"bdec_version":3}],"role":"testrole_kafka","request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_2"} -18-08-2023 12:45:40 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/19/45/rzlqw4_WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -18-08-2023 12:45:41 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 -18-08-2023 12:45:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 12:45:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1052 -18-08-2023 12:45:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:45:44 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 1 file names -18-08-2023 12:45:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 12:45:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:45:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:46 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -18-08-2023 12:45:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 12:45:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 12:45:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:49 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 12:45:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 12:45:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:51 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:45:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 12:45:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 12:45:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 12:45:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:54 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:45:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 12:45:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 -18-08-2023 12:45:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:56 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:45:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 12:45:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 12:45:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 -18-08-2023 12:45:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:45:59 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:46:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 12:46:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 12:46:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:01 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:46:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 12:46:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 -18-08-2023 12:46:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:04 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:46:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 12:46:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 12:46:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:06 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:46:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 12:46:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 12:46:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 12:46:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:09 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:46:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 12:46:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:46:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -18-08-2023 12:46:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:10 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0 -18-08-2023 12:46:10 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:46:10 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, offset:1 -18-08-2023 12:46:10 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 12:46:10 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 12:46:10 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"WKBBd3sm9vwmqMs3Unz3ZaM2WTh9ZKBBTm0DPIe6RlpFsCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4720053319416705063","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 12:46:11 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 12:46:11 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 12:46:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=527, currentDiffMillis=527 -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=8, mean=0.07917226020632413, stddev=0.6470555470389748, p50=0.0, p75=0.0, p95=0.0, p98=1.0, p99=4.0, p999=8.0 -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.422444063774894, rate_unit=events/second -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.738717882934, rate_unit=events/second -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=8.852083, max=8.852083, mean=8.852083, stddev=0.0, p50=8.852083, p75=8.852083, p95=8.852083, p98=8.852083, p99=8.852083, p999=8.852083, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03145986490791663, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=371.17925, max=371.17925, mean=371.17925, stddev=0.0, p50=371.17925, p75=371.17925, p95=371.17925, p98=371.17925, p99=371.17925, p999=371.17925, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0314596827177743, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=162.694458, max=162.694458, mean=162.694458, stddev=0.0, p50=162.694458, p75=162.694458, p95=162.694458, p98=162.694458, p99=162.694458, p999=162.694458, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031459540653228084, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:46:11 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=198.545875, max=198.545875, mean=198.545875, stddev=0.0, p50=198.545875, p75=198.545875, p95=198.545875, p98=198.545875, p99=198.545875, p999=198.545875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03145935793141843, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 12:46:11 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:46:11 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:46:11 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 12:46:11 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:46:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:46:11 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:46:12 pool-4-thread-1 INFO RequestBuilder:458 - Final History URIBuilder - https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0/insertReport?requestId=2019ea59-9641-49e5-99d5-36b56c767dcb -18-08-2023 12:46:12 pool-4-thread-1 INFO SimpleIngestManager:601 - Attempting to unmarshall history response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:46:12 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Connection: keep-alive] net.snowflake.client.jdbc.internal.apache.http.client.entity.DecompressingEntity@75b2ff9} -18-08-2023 12:46:12 pool-4-thread-1 INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] searched 1 files in ingest report, found 1 -18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] Purging loaded files for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0, loadedFileCount:1, loadedFiles:[TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz] -18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] deleted TEST_CONNECTOR/kafka_connector_test_table_8383920415137154704/0/0_2_1692387912137.json.gz from stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:46:12 pool-4-thread-1 INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] purge 1 files from stage: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 -18-08-2023 12:46:12 pool-4-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":5492,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":59991,"average_ingestion_lag_file_count":1,"average_commit_lag":1356,"average_commit_lag_file_count":1,"start_time":1692387911337,"end_time":1692387972798,"is_pipe_closing":false} of type:kafka_pipe_usage -18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 retrieved 0 file names -18-08-2023 12:46:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: cleaner terminated -18-08-2023 12:46:14 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:46:14 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:46:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:46:14 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:46:14 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8383920415137154704","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0","processed-offset":2,"flushed-offset":2,"committed-offset":2,"purged-offset":2,"record_number":3,"byte_number":5492,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":1,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692387972798,"end_time":1692387974518,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:46:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0: service closed -18-08-2023 12:46:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 -18-08-2023 12:46:14 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 is:0, names:[] -18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8383920415137154704 dropped -18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_0 dropped -18-08-2023 12:46:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8383920415137154704_1 dropped -18-08-2023 12:46:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3264218329753027732 -18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3264218329753027732_0 -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3264218329753027732 dropped -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3264218329753027732_0 dropped -18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9072147538436586285 -18-08-2023 12:47:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9072147538436586285_0 -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9072147538436586285 dropped -18-08-2023 12:47:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9072147538436586285_0 dropped -18-08-2023 12:47:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3050797644145311050 -18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3050797644145311050_0 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3050797644145311050 dropped -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3050797644145311050_0 dropped -18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2363792425311133864 -18-08-2023 12:47:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2363792425311133864_0 -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2363792425311133864 dropped -18-08-2023 12:47:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2363792425311133864_0 dropped -18-08-2023 12:47:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:47:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:47:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:48:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:48:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:48:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:48:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 -18-08-2023 12:48:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8546433757402467292 -18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:48:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 -18-08-2023 12:48:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:30 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:48:30 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 12:48:30 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:48:30 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:48:30 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:48:30 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:30 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, existing:[] -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 - service started -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8546433757402467292, topicPartition: test-0 -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8546433757402467292. -18-08-2023 12:48:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 retrieved 0 file names -18-08-2023 12:48:30 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292. -18-08-2023 12:48:30 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 doesn't exist -18-08-2023 12:48:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:48:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 retrieved 0 file names -18-08-2023 12:48:31 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: cleaner started -18-08-2023 12:48:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8546433757402467292","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388110425} of type:kafka_pipe_start -18-08-2023 12:48:31 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz -18-08-2023 12:48:31 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 12:48:31 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292, filePath:TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz -18-08-2023 12:48:31 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 -18-08-2023 12:48:31 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292, filePath:TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz -18-08-2023 12:48:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz -18-08-2023 12:48:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz] -18-08-2023 12:48:31 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8546433757402467292/0/0_0_1692388111184.json.gz] -18-08-2023 12:48:31 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0/insertFiles?requestId=6ff1ca31-015a-4a4d-889b-910b09eee40c&showSkippedFiles=false -18-08-2023 12:48:32 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 19:48:32 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 12:48:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:48:35 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:48:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:48:45 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:48:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:48:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:49:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:49:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: cleaner terminated -18-08-2023 12:49:03 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 12:49:03 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 12:49:03 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 12:49:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 12:49:03 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8546433757402467292","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1278,"average_commit_lag_file_count":1,"start_time":1692388110417,"end_time":1692388143128,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 12:49:03 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0: service closed -18-08-2023 12:49:03 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 -18-08-2023 12:49:03 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 is:0, names:[] -18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8546433757402467292 dropped -18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8546433757402467292_0 dropped -18-08-2023 12:49:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 12:49:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 12:49:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:04 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 12:49:04 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 12:49:04 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 12:49:04 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 12:49:04 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:04 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0, existing:[] -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 - service started -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_515941071091186449, topicPartition: test-0 -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 12:49:04 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 12:49:04 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 12:49:04 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_515941071091186449. -18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 retrieved 0 file names -18-08-2023 12:49:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449. -18-08-2023 12:49:04 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 doesn't exist -18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0 -18-08-2023 12:49:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449 retrieved 0 file names -18-08-2023 12:49:04 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0: cleaner started -18-08-2023 12:49:04 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388144104} of type:kafka_pipe_start -18-08-2023 12:49:04 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_key_1692388144825.gz -18-08-2023 12:49:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_key_1692388144825.gz to table stage: kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:05 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_value_1692388145170.gz -18-08-2023 12:49:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_515941071091186449/0/1_value_1692388145170.gz to table stage: kafka_connector_test_table_515941071091186449 -18-08-2023 12:49:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:49:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:49:59 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:04 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:04 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692388144015,"end_time":1692388204837,"is_pipe_closing":false} of type:kafka_pipe_usage -18-08-2023 12:50:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:50:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:14 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:19 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:50:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:50:59 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:51:04 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 12:51:04 pool-7-thread-1 DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_515941071091186449","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_515941071091186449","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_515941071091186449_0","processed-offset":-1,"flushed-offset":-1,"committed-offset":-1,"purged-offset":-1,"record_number":0,"byte_number":0,"file-count-on-stage":0,"file-count-on-ingestion":0,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":2,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":0,"average_commit_lag_file_count":0,"start_time":1692388204837,"end_time":1692388264835,"is_pipe_closing":false} of type:kafka_pipe_usage -18-08-2023 12:51:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 12:51:09 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:00:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 -18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4136623387971130024 -18-08-2023 13:00:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 -18-08-2023 13:00:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:24 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:00:24 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:00:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:00:24 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:00:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:00:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, existing:[] -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 - service started -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4136623387971130024, topicPartition: test-0 -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4136623387971130024. -18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 retrieved 0 file names -18-08-2023 13:00:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024. -18-08-2023 13:00:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 doesn't exist -18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 retrieved 0 file names -18-08-2023 13:00:25 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: cleaner started -18-08-2023 13:00:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4136623387971130024","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388824753} of type:kafka_pipe_start -18-08-2023 13:00:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz -18-08-2023 13:00:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:00:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024, filePath:TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz -18-08-2023 13:00:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 -18-08-2023 13:00:26 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024, filePath:TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz -18-08-2023 13:00:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz -18-08-2023 13:00:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz] -18-08-2023 13:00:26 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4136623387971130024/0/0_0_1692388825543.json.gz] -18-08-2023 13:00:26 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0/insertFiles?requestId=2c223d0a-7b31-49c4-9c07-73804a59d4c2&showSkippedFiles=false -18-08-2023 13:00:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:00:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:00:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:29 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:34 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:39 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:49 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:54 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:00:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: cleaner terminated -18-08-2023 13:00:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:00:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:00:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:00:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4136623387971130024","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1254,"average_commit_lag_file_count":1,"start_time":1692388824745,"end_time":1692388857570,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:00:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0: service closed -18-08-2023 13:00:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 -18-08-2023 13:00:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 is:0, names:[] -18-08-2023 13:00:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4136623387971130024 dropped -18-08-2023 13:00:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4136623387971130024_0 dropped -18-08-2023 13:00:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 -18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7116658905857989900 -18-08-2023 13:00:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 -18-08-2023 13:00:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:58 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:00:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:00:58 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:00:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:00:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0, existing:[] -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 - service started -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7116658905857989900, topicPartition: test-0 -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:00:58 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 13:00:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:00:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7116658905857989900. -18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 retrieved 0 file names -18-08-2023 13:00:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900. -18-08-2023 13:00:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 doesn't exist -18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0 -18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900 retrieved 0 file names -18-08-2023 13:00:59 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0: cleaner started -18-08-2023 13:00:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7116658905857989900","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7116658905857989900","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7116658905857989900_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388858714} of type:kafka_pipe_start -18-08-2023 13:00:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_key_1692388859543.gz -18-08-2023 13:00:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_key_1692388859543.gz to table stage: kafka_connector_test_table_7116658905857989900 -18-08-2023 13:00:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_value_1692388859956.gz -18-08-2023 13:01:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_7116658905857989900/0/1_value_1692388859956.gz to table stage: kafka_connector_test_table_7116658905857989900 -18-08-2023 13:01:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:01:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:01:33 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:01:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:01:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:01:49 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:01:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 -18-08-2023 13:01:49 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4080866019933655485 -18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:01:50 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 -18-08-2023 13:01:50 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:50 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:01:50 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:01:50 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:01:50 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:01:50 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:01:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:50 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, existing:[] -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 - service started -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4080866019933655485, topicPartition: test-0 -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4080866019933655485. -18-08-2023 13:01:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 retrieved 0 file names -18-08-2023 13:01:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485. -18-08-2023 13:01:50 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 doesn't exist -18-08-2023 13:01:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:01:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 retrieved 0 file names -18-08-2023 13:01:51 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: cleaner started -18-08-2023 13:01:51 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4080866019933655485","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388910373} of type:kafka_pipe_start -18-08-2023 13:01:51 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz -18-08-2023 13:01:51 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:01:51 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485, filePath:TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz -18-08-2023 13:01:51 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 -18-08-2023 13:01:51 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485, filePath:TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz -18-08-2023 13:01:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz -18-08-2023 13:01:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz] -18-08-2023 13:01:51 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_4080866019933655485/0/0_0_1692388911253.json.gz] -18-08-2023 13:01:51 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0/insertFiles?requestId=397783b8-a935-4025-964d-58f6bc46aff0&showSkippedFiles=false -18-08-2023 13:01:52 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:01:52 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:01:52 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:01:55 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:05 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:15 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:20 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:23 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:02:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: cleaner terminated -18-08-2023 13:02:23 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:02:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:02:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:02:23 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4080866019933655485","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1266,"average_commit_lag_file_count":1,"start_time":1692388910365,"end_time":1692388943363,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:02:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0: service closed -18-08-2023 13:02:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 -18-08-2023 13:02:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 is:0, names:[] -18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4080866019933655485 dropped -18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4080866019933655485_0 dropped -18-08-2023 13:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:02:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:02:24 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:02:24 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:02:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:02:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0, existing:[] -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 - service started -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_487147896604185713, topicPartition: test-0 -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:02:24 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 13:02:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:02:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_487147896604185713. -18-08-2023 13:02:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 retrieved 0 file names -18-08-2023 13:02:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713. -18-08-2023 13:02:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 doesn't exist -18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0 -18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713 retrieved 0 file names -18-08-2023 13:02:25 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0: cleaner started -18-08-2023 13:02:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_487147896604185713","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_487147896604185713","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_487147896604185713_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388944320} of type:kafka_pipe_start -18-08-2023 13:02:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_key_1692388945210.gz -18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_key_1692388945210.gz to table stage: kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_value_1692388945600.gz -18-08-2023 13:02:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_487147896604185713/0/1_value_1692388945600.gz to table stage: kafka_connector_test_table_487147896604185713 -18-08-2023 13:02:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:29 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:34 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:39 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:44 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:49 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:54 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:54 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:54 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:02:54 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:02:54 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:02:54 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:02:54 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:02:54 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:54 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0, existing:[] -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 - service started -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5916591702771815345, topicPartition: test-0 -18-08-2023 13:02:54 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:02:55 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 13:02:55 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:02:55 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5916591702771815345. -18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 retrieved 0 file names -18-08-2023 13:02:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345. -18-08-2023 13:02:55 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 doesn't exist -18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0 -18-08-2023 13:02:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345 retrieved 0 file names -18-08-2023 13:02:55 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0: cleaner started -18-08-2023 13:02:55 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5916591702771815345","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5916591702771815345","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5916591702771815345_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692388975041} of type:kafka_pipe_start -18-08-2023 13:02:55 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_key_1692388975848.gz -18-08-2023 13:02:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:02:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_key_1692388975848.gz to table stage: kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:56 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_value_1692388976385.gz -18-08-2023 13:02:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5916591702771815345/0/1_value_1692388976385.gz to table stage: kafka_connector_test_table_5916591702771815345 -18-08-2023 13:02:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:03:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:03:01 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:03:02 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:02 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:37 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:03:37 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:03:37 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:03:37 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:03:37 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:03:37 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:37 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0, existing:[] -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 - service started -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4425459254925033856, topicPartition: test-0 -18-08-2023 13:03:37 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:03:38 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 13:03:38 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:03:38 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:03:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:03:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_4425459254925033856. -18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 retrieved 0 file names -18-08-2023 13:03:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856. -18-08-2023 13:03:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 doesn't exist -18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0 -18-08-2023 13:03:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856 retrieved 0 file names -18-08-2023 13:03:43 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0: cleaner started -18-08-2023 13:03:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_4425459254925033856","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4425459254925033856","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4425459254925033856_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692389022721} of type:kafka_pipe_start -18-08-2023 13:03:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_key_1692389023506.gz -18-08-2023 13:03:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_key_1692389023506.gz to table stage: kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:44 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_value_1692389024041.gz -18-08-2023 13:03:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_4425459254925033856/0/1_value_1692389024041.gz to table stage: kafka_connector_test_table_4425459254925033856 -18-08-2023 13:03:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:03:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:03:52 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:03:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:02 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:07 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:12 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:17 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:04:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:04:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:04:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:04:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:04:52 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:52 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:04:53 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:53 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:53 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:04:53 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:04:53 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:04:53 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:04:53 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:04:53 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:53 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0, existing:[] -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 - service started -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5251568966378274834, topicPartition: test-0 -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:04:53 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 13:04:53 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:04:53 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_5251568966378274834. -18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 retrieved 0 file names -18-08-2023 13:04:53 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834. -18-08-2023 13:04:53 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 doesn't exist -18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0 -18-08-2023 13:04:53 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834 retrieved 0 file names -18-08-2023 13:04:53 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0: cleaner started -18-08-2023 13:04:53 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_5251568966378274834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5251568966378274834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5251568966378274834_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692389093264} of type:kafka_pipe_start -18-08-2023 13:04:53 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_key_1692389093942.gz -18-08-2023 13:04:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_key_1692389093942.gz to table stage: kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:54 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_value_1692389094485.gz -18-08-2023 13:04:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] put file: TEST_CONNECTOR/kafka_connector_test_table_5251568966378274834/0/1_value_1692389094485.gz to table stage: kafka_connector_test_table_5251568966378274834 -18-08-2023 13:04:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:04:58 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:13 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:18 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:23 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:05:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:05:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:06:51 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:07:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:07:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:07:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:07:03 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:07:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 -18-08-2023 13:07:03 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 -18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_304205135411409227 -18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:07:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 -18-08-2023 13:07:04 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 -18-08-2023 13:07:04 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:07:04 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:07:04 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:07:04 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:07:04 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:07:04 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 -18-08-2023 13:07:04 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0, existing:[] -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 - service started -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_304205135411409227, topicPartition: test-0 -18-08-2023 13:07:04 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_304205135411409227 dropped -18-08-2023 13:07:04 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_304205135411409227_0 dropped -18-08-2023 13:07:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:08:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:08:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:08:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:08:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 -18-08-2023 13:08:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 -18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5819610730455116141 -18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:08:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 -18-08-2023 13:08:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 -18-08-2023 13:08:32 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:08:32 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:08:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:08:32 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:08:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:08:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 -18-08-2023 13:08:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0, existing:[] -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 - service started -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5819610730455116141, topicPartition: test-0 -18-08-2023 13:08:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5819610730455116141 dropped -18-08-2023 13:08:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5819610730455116141_0 dropped -18-08-2023 13:08:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:10:55 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:10:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 -18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 -18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7486010370272619727 -18-08-2023 13:10:56 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 -18-08-2023 13:10:56 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 -18-08-2023 13:10:56 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:10:56 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:10:56 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:10:56 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:10:56 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:10:56 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 -18-08-2023 13:10:56 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0, existing:[] -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 - service started -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7486010370272619727, topicPartition: test-0 -18-08-2023 13:10:56 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:10:56 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:10:56 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:10:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:10:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0: cleaner terminated -18-08-2023 13:10:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:10:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:10:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0: service closed -18-08-2023 13:10:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 13:10:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 -18-08-2023 13:10:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 is:0, names:[] -18-08-2023 13:10:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7486010370272619727 dropped -18-08-2023 13:10:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7486010370272619727_0 dropped -18-08-2023 13:10:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:17:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:17:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 -18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 -18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3983027369971842529 -18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 -18-08-2023 13:17:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 -18-08-2023 13:17:08 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:17:08 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:17:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:17:08 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:17:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:17:08 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 -18-08-2023 13:17:08 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0, existing:[] -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 - service started -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3983027369971842529, topicPartition: test-0 -18-08-2023 13:17:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3983027369971842529 dropped -18-08-2023 13:17:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3983027369971842529_0 dropped -18-08-2023 13:17:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:21:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2327466389014140672 -18-08-2023 13:21:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2327466389014140672_0 -18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2327466389014140672 dropped -18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2327466389014140672_0 dropped -18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:27 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:21:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:21:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 -18-08-2023 13:21:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 -18-08-2023 13:21:37 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:37 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. -18-08-2023 13:21:37 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:37 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:37 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1932307370833503227 -18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:21:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 -18-08-2023 13:21:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 -18-08-2023 13:21:38 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:21:38 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:21:38 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:21:38 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:21:38 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:21:38 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 -18-08-2023 13:21:38 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0, existing:[] -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 - service started -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1932307370833503227, topicPartition: test-0 -18-08-2023 13:21:38 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:21:38 main DEBUG RestService:260 - Sending POST with input {"schema":"{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"int16\",\"type\":{\"type\":\"int\",\"connect.type\":\"int16\"}}]}"} to http://fake-url/subjects/test-value/versions?normalize=false -18-08-2023 13:21:38 main ERROR RestService:276 - Failed to send HTTP request to endpoint: http://fake-url/subjects/test-value/versions?normalize=false -java.net.UnknownHostException: fake-url - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:229) - at java.base/java.net.Socket.connect(Socket.java:609) - at java.base/sun.net.NetworkClient.doConnect(NetworkClient.java:177) - at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:508) - at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:603) - at java.base/sun.net.www.http.HttpClient.(HttpClient.java:276) - at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:375) - at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:396) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1253) - at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1187) - at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:1081) - at java.base/sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:1015) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream0(HttpURLConnection.java:1367) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream(HttpURLConnection.java:1342) - at io.confluent.kafka.schemaregistry.client.rest.RestService.sendHttpRequest(RestService.java:272) - at io.confluent.kafka.schemaregistry.client.rest.RestService.httpRequest(RestService.java:371) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:548) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:536) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:494) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.registerAndGetId(CachedSchemaRegistryClient.java:274) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:381) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:354) - at io.confluent.kafka.serializers.AbstractKafkaAvroSerializer.serializeImpl(AbstractKafkaAvroSerializer.java:125) - at io.confluent.connect.avro.AvroConverter$Serializer.serialize(AvroConverter.java:153) - at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:86) - at com.snowflake.kafka.connector.internal.TombstoneRecordIngestionIT.testIgnoreTombstoneRecordBehavior(TombstoneRecordIngestionIT.java:208) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1932307370833503227 dropped -18-08-2023 13:21:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1932307370833503227_0 dropped -18-08-2023 13:21:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:39 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:21:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:21:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:21:39 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:21:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:21:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:21:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:21:39 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:21:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:21:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:21:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:21:39 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -18-08-2023 13:21:39 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:21:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:21:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:21:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:21:39 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:21:39 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_6031348541095650961 doesn't exist -18-08-2023 13:21:39 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_6031348541095650961. -18-08-2023 13:21:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6031348541095650961 -18-08-2023 13:21:39 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_6031348541095650961 -18-08-2023 13:21:39 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6031348541095650961, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:21:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"wKkrlehYLnc2nFjUQjRSPOP5YQ7uuMMajhbOAAYHOIgcCC_1003_0","table":"kafka_connector_test_table_6031348541095650961"} -18-08-2023 13:21:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_6031348541095650961, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:21:40 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961 -18-08-2023 13:21:40 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:21:40 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0 -18-08-2023 13:21:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"wKkrlehYLnc2nFjUQjRSPOP5YQ7uuMMajhbOAAYHOIgcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:21:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, offset:null -18-08-2023 13:21:40 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_6031348541095650961.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:21:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6031348541095650961, topicPartition: test-0 -18-08-2023 13:21:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:21:40 main DEBUG RestService:260 - Sending POST with input {"schema":"{\"type\":\"record\",\"name\":\"ConnectDefault\",\"namespace\":\"io.confluent.connect.avro\",\"fields\":[{\"name\":\"int16\",\"type\":{\"type\":\"int\",\"connect.type\":\"int16\"}}]}"} to http://fake-url/subjects/test-value/versions?normalize=false -18-08-2023 13:21:40 main ERROR RestService:276 - Failed to send HTTP request to endpoint: http://fake-url/subjects/test-value/versions?normalize=false -java.net.UnknownHostException: fake-url - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:229) - at java.base/java.net.Socket.connect(Socket.java:609) - at java.base/sun.net.NetworkClient.doConnect(NetworkClient.java:177) - at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:508) - at java.base/sun.net.www.http.HttpClient.openServer(HttpClient.java:603) - at java.base/sun.net.www.http.HttpClient.(HttpClient.java:276) - at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:375) - at java.base/sun.net.www.http.HttpClient.New(HttpClient.java:396) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1253) - at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1187) - at java.base/sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:1081) - at java.base/sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:1015) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream0(HttpURLConnection.java:1367) - at java.base/sun.net.www.protocol.http.HttpURLConnection.getOutputStream(HttpURLConnection.java:1342) - at io.confluent.kafka.schemaregistry.client.rest.RestService.sendHttpRequest(RestService.java:272) - at io.confluent.kafka.schemaregistry.client.rest.RestService.httpRequest(RestService.java:371) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:548) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:536) - at io.confluent.kafka.schemaregistry.client.rest.RestService.registerSchema(RestService.java:494) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.registerAndGetId(CachedSchemaRegistryClient.java:274) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:381) - at io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.register(CachedSchemaRegistryClient.java:354) - at io.confluent.kafka.serializers.AbstractKafkaAvroSerializer.serializeImpl(AbstractKafkaAvroSerializer.java:125) - at io.confluent.connect.avro.AvroConverter$Serializer.serialize(AvroConverter.java:153) - at io.confluent.connect.avro.AvroConverter.fromConnectData(AvroConverter.java:86) - at com.snowflake.kafka.connector.internal.TombstoneRecordIngestionIT.testIgnoreTombstoneRecordBehavior(TombstoneRecordIngestionIT.java:208) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -18-08-2023 13:21:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:25:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 -18-08-2023 13:25:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:13 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:25:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7700526656325775771 -18-08-2023 13:25:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:25:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 -18-08-2023 13:25:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:14 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:25:14 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:25:14 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:25:14 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:25:14 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:25:14 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:14 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, existing:[] -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 - service started -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7700526656325775771, topicPartition: test-0 -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:25:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:25:14 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7700526656325775771. -18-08-2023 13:25:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 retrieved 0 file names -18-08-2023 13:25:14 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771. -18-08-2023 13:25:14 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 doesn't exist -18-08-2023 13:25:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 retrieved 0 file names -18-08-2023 13:25:15 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: cleaner started -18-08-2023 13:25:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7700526656325775771","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390314362} of type:kafka_pipe_start -18-08-2023 13:25:15 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz -18-08-2023 13:25:15 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 -18-08-2023 13:25:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771, filePath:TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz -18-08-2023 13:25:15 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 -18-08-2023 13:25:15 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771, filePath:TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz -18-08-2023 13:25:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz -18-08-2023 13:25:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz] -18-08-2023 13:25:15 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7700526656325775771/0/1_1_1692390315175.json.gz] -18-08-2023 13:25:15 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0/insertFiles?requestId=648c4ee4-096c-4e1a-bd5f-a3e13508a1be&showSkippedFiles=false -18-08-2023 13:25:16 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:25:16 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:25:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:19 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:24 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:29 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:34 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:39 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:47 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:25:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: cleaner terminated -18-08-2023 13:25:47 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:25:47 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:25:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:47 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7700526656325775771","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1306,"average_commit_lag_file_count":1,"start_time":1692390314321,"end_time":1692390347574,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:25:47 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0: service closed -18-08-2023 13:25:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 -18-08-2023 13:25:47 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 is:0, names:[] -18-08-2023 13:25:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7700526656325775771 dropped -18-08-2023 13:25:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7700526656325775771_0 dropped -18-08-2023 13:25:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:25:48 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:48 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:48 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:25:48 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:25:48 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:25:48 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:25:48 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:48 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:25:48 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:25:48 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:25:48 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:25:48 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:25:48 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:25:48 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -18-08-2023 13:25:48 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:25:48 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:25:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:25:48 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:25:48 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:25:48 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4977398750416568100 doesn't exist -18-08-2023 13:25:48 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4977398750416568100. -18-08-2023 13:25:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4977398750416568100 -18-08-2023 13:25:48 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4977398750416568100 -18-08-2023 13:25:48 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4977398750416568100, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:25:48 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_0","table":"kafka_connector_test_table_4977398750416568100"} -18-08-2023 13:25:49 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4977398750416568100, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:25:49 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100 -18-08-2023 13:25:49 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:49 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset:null -18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:25:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4977398750416568100, topicPartition: test-0 -18-08-2023 13:25:49 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:25:49 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:25:49 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0,currentBufferSizeInBytes:211, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1} -18-08-2023 13:25:49 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 211 bytes, offset 1 - 1 -18-08-2023 13:25:49 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 13:25:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:25:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1038 -18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 -18-08-2023 13:25:49 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1a54a74d[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec] -18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 13:25:49 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, idx=0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. -18-08-2023 13:25:49 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 105, 110, 116, 49, 54, 34, 58, 49, 50, 125] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(12 bytes) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 16 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 16 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 40 to byteArray of 40 bytes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 40 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 63 to byteArray of 63 bytes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 168: end column -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 63B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 22B raw, 40B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 168: end block -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 168: column indexes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 330: offset indexes -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 352: bloom filters -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 352: end -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1088: footer length = 736 -18-08-2023 13:25:49 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 736 => 224 2 0 0 -18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100, rowCount=1, startOffset=0, estimatedUncompressedSize=74.5, paddedChunkLength=1096, encryptedCompressedSize=1104, bdecVersion=THREE -18-08-2023 13:25:49 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, size=1104 -18-08-2023 13:25:50 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, size=1104, timeInMillis=226 -18-08-2023 13:25:50 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec, idx=0 -18-08-2023 13:25:50 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 13:25:50 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 13:25:50 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec","md5":"5e965e0219d5aee662bf744dca2310b8","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","chunk_start_offset":0,"chunk_length":1096,"chunk_length_uncompressed":74,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"5c1f25772925ad600e144c2f604d1e6c","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370812214,"first_insert_time_in_ms":1692390349190,"last_insert_time_in_ms":1692390349190,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390349512,"build_duration_ms":269,"upload_duration_ms":223},"bdec_version":3}],"role":"testrole_kafka","request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_2"} -18-08-2023 13:25:50 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/25/rzlsr1_Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_26_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 13:25:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 -18-08-2023 13:25:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -18-08-2023 13:25:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 13:25:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:53 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 -18-08-2023 13:25:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1043 -18-08-2023 13:25:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:25:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:25:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -18-08-2023 13:25:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:58 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:25:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1051 -18-08-2023 13:25:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:25:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 13:25:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1043 -18-08-2023 13:26:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 13:26:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 13:26:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:26:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 -18-08-2023 13:26:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:26:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 13:26:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:26:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:26:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:26:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 -18-08-2023 13:26:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1048 -18-08-2023 13:26:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:26:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:26:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:26:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 -18-08-2023 13:26:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1028 -18-08-2023 13:26:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 13:26:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1029 -18-08-2023 13:26:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -18-08-2023 13:26:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:26:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:26:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:26:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:26:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0 -18-08-2023 13:26:20 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:26:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, offset:1 -18-08-2023 13:26:20 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 13:26:20 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"Lub674afHxtKASZL3FO6dNUGrodgCgYZmpKFYRJyTBcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4977398750416568100","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:26:20 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 13:26:20 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 13:26:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=545, currentDiffMillis=545 -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1104, max=1104, mean=1104.0, stddev=0.0, p50=1104.0, p75=1104.0, p95=1104.0, p98=1104.0, p99=1104.0, p999=1104.0 -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=18, mean=0.3540181330563173, stddev=1.9300136683346523, p50=0.0, p75=0.0, p95=1.0, p98=8.0, p99=11.0, p999=18.0 -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=74, m1_rate=9.75676132696657, m5_rate=13.616657336513985, m15_rate=14.394546261321954, mean_rate=2.3148080625671974, rate_unit=events/second -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1104, m1_rate=145.560331148258, m5_rate=203.14580675015458, m15_rate=214.7510685472897, mean_rate=34.534186912207815, rate_unit=events/second -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=269.496, max=269.496, mean=269.496, stddev=0.0, p50=269.496, p75=269.496, p95=269.496, p98=269.496, p99=269.496, p999=269.496, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031279512342080346, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=639.33075, max=639.33075, mean=639.33075, stddev=0.0, p50=639.33075, p75=639.33075, p95=639.33075, p98=639.33075, p99=639.33075, p999=639.33075, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127928580120086, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=139.9175, max=139.9175, mean=139.9175, stddev=0.0, p50=139.9175, p75=139.9175, p95=139.9175, p98=139.9175, p99=139.9175, p999=139.9175, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03127915502279425, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:26:20 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=223.235042, max=223.235042, mean=223.235042, stddev=0.0, p50=223.235042, p75=223.235042, p95=223.235042, p98=223.235042, p99=223.235042, p999=223.235042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031278966155411835, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:26:20 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:26:20 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:26:20 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:26:20 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:26:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 -18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8604860857340579093 -18-08-2023 13:29:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 -18-08-2023 13:29:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:23 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:29:23 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 13:29:23 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:29:23 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:29:23 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:29:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:23 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, existing:[] -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 - service started -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8604860857340579093, topicPartition: test-0 -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8604860857340579093. -18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 retrieved 0 file names -18-08-2023 13:29:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093. -18-08-2023 13:29:24 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 doesn't exist -18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 retrieved 0 file names -18-08-2023 13:29:24 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: cleaner started -18-08-2023 13:29:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8604860857340579093","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390563717} of type:kafka_pipe_start -18-08-2023 13:29:24 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz -18-08-2023 13:29:24 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:29:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093, filePath:TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz -18-08-2023 13:29:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 -18-08-2023 13:29:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093, filePath:TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz -18-08-2023 13:29:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz -18-08-2023 13:29:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz] -18-08-2023 13:29:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8604860857340579093/0/0_0_1692390564848.json.gz] -18-08-2023 13:29:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0/insertFiles?requestId=1359fcb3-ffcd-4e13-888e-46010696cb74&showSkippedFiles=false -18-08-2023 13:29:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:29:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:29:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:28 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:33 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:38 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:43 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:53 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:57 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:29:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: cleaner terminated -18-08-2023 13:29:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:29:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:29:57 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:29:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8604860857340579093","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1362,"average_commit_lag_file_count":1,"start_time":1692390563709,"end_time":1692390597754,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:29:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0: service closed -18-08-2023 13:29:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 -18-08-2023 13:29:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 is:0, names:[] -18-08-2023 13:29:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8604860857340579093 dropped -18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8604860857340579093_0 dropped -18-08-2023 13:29:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 -18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6937783810298395356 -18-08-2023 13:29:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 -18-08-2023 13:29:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:58 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:29:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:29:58 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:29:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:29:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:58 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, existing:[] -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 - service started -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6937783810298395356, topicPartition: test-0 -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:29:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value schema is:com.snowflake.kafka.connector.records.SnowflakeJsonSchema and value is Empty Json Node for topic test, partition 0 and offset 0 -18-08-2023 13:29:58 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6937783810298395356. -18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 retrieved 0 file names -18-08-2023 13:29:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356. -18-08-2023 13:29:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 doesn't exist -18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:29:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 retrieved 0 file names -18-08-2023 13:29:59 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: cleaner started -18-08-2023 13:29:59 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6937783810298395356","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390598810} of type:kafka_pipe_start -18-08-2023 13:29:59 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz -18-08-2023 13:29:59 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 13:29:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356, filePath:TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz -18-08-2023 13:29:59 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 -18-08-2023 13:30:00 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356, filePath:TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz -18-08-2023 13:30:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz -18-08-2023 13:30:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz] -18-08-2023 13:30:00 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6937783810298395356/0/1_1_1692390599529.json.gz] -18-08-2023 13:30:00 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0/insertFiles?requestId=4f4a34a6-88cf-4d09-8536-db7d4f800c68&showSkippedFiles=false -18-08-2023 13:30:00 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:30:00 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:30:00 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:03 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:08 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:13 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:18 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:23 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:28 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: cleaner terminated -18-08-2023 13:30:31 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:30:31 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:30:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:30:31 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:31 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6937783810298395356","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":983,"average_commit_lag_file_count":1,"start_time":1692390598807,"end_time":1692390631388,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:30:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0: service closed -18-08-2023 13:30:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 -18-08-2023 13:30:31 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 is:0, names:[] -18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6937783810298395356 dropped -18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6937783810298395356_0 dropped -18-08-2023 13:30:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 -18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1197281858695507582 -18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 -18-08-2023 13:30:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:32 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:30:32 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:30:32 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:30:32 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:30:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:32 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, existing:[] -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 - service started -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1197281858695507582, topicPartition: test-0 -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1197281858695507582. -18-08-2023 13:30:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 retrieved 0 file names -18-08-2023 13:30:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582. -18-08-2023 13:30:32 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 doesn't exist -18-08-2023 13:30:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:30:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 retrieved 0 file names -18-08-2023 13:30:33 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: cleaner started -18-08-2023 13:30:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1197281858695507582","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390632470} of type:kafka_pipe_start -18-08-2023 13:30:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz -18-08-2023 13:30:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:30:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582, filePath:TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz -18-08-2023 13:30:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 -18-08-2023 13:30:33 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582, filePath:TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz -18-08-2023 13:30:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz -18-08-2023 13:30:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz] -18-08-2023 13:30:33 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1197281858695507582/0/0_0_1692390633273.json.gz] -18-08-2023 13:30:33 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0/insertFiles?requestId=2ca659be-2980-4719-9763-a479c83040d4&showSkippedFiles=false -18-08-2023 13:30:34 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:30:34 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:30:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:30:37 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:42 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:47 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:52 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:30:57 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:02 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:04 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: cleaner terminated -18-08-2023 13:31:05 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:31:05 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:31:05 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:31:05 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:05 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1197281858695507582","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":984,"average_commit_lag_file_count":1,"start_time":1692390632467,"end_time":1692390665074,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:31:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0: service closed -18-08-2023 13:31:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 -18-08-2023 13:31:05 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 is:0, names:[] -18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1197281858695507582 dropped -18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1197281858695507582_0 dropped -18-08-2023 13:31:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:31:05 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 -18-08-2023 13:31:05 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:05 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3765566255072832355 -18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:31:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 -18-08-2023 13:31:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:31:06 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:31:06 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:31:06 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:31:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:06 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, existing:[] -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 - service started -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3765566255072832355, topicPartition: test-0 -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:31:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:31:06 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3765566255072832355. -18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 retrieved 0 file names -18-08-2023 13:31:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355. -18-08-2023 13:31:06 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 doesn't exist -18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 retrieved 0 file names -18-08-2023 13:31:06 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: cleaner started -18-08-2023 13:31:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3765566255072832355","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390666153} of type:kafka_pipe_start -18-08-2023 13:31:06 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz -18-08-2023 13:31:06 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 13:31:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355, filePath:TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz -18-08-2023 13:31:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 -18-08-2023 13:31:07 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355, filePath:TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz -18-08-2023 13:31:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz -18-08-2023 13:31:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz] -18-08-2023 13:31:07 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3765566255072832355/0/1_1_1692390666997.json.gz] -18-08-2023 13:31:07 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0/insertFiles?requestId=1e4d8fab-60bc-495a-9788-5cb8b1b71bb3&showSkippedFiles=false -18-08-2023 13:31:09 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:31:09 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:31:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:11 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:16 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:21 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:26 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:31 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:36 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: cleaner terminated -18-08-2023 13:31:40 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:31:40 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:31:40 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:31:40 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:40 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3765566255072832355","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":2333,"average_commit_lag_file_count":1,"start_time":1692390666140,"end_time":1692390700233,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:31:40 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0: service closed -18-08-2023 13:31:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 -18-08-2023 13:31:40 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 is:0, names:[] -18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3765566255072832355 dropped -18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3765566255072832355_0 dropped -18-08-2023 13:31:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:31:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 -18-08-2023 13:31:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 -18-08-2023 13:31:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:40 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. -18-08-2023 13:31:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:40 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3449896833657992937 -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 -18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 -18-08-2023 13:31:41 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:31:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:31:41 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:31:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:31:41 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 -18-08-2023 13:31:41 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0, existing:[] -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 - service started -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3449896833657992937, topicPartition: test-0 -18-08-2023 13:31:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3449896833657992937 dropped -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3449896833657992937_0 dropped -18-08-2023 13:31:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 -18-08-2023 13:31:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 -18-08-2023 13:31:41 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:41 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:41 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:31:41 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:31:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_188487355215212289 -18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 -18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 -18-08-2023 13:31:42 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:31:42 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:31:42 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:31:42 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 -18-08-2023 13:31:42 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0, existing:[] -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 - service started -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_188487355215212289, topicPartition: test-0 -18-08-2023 13:31:42 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_188487355215212289 dropped -18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_188487355215212289_0 dropped -18-08-2023 13:31:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 -18-08-2023 13:31:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:42 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1080103072068495127 -18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:31:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 -18-08-2023 13:31:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:43 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:31:43 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:31:43 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:31:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:43 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, existing:[] -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 - service started -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1080103072068495127, topicPartition: test-0 -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1080103072068495127. -18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 retrieved 0 file names -18-08-2023 13:31:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127. -18-08-2023 13:31:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 doesn't exist -18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:31:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 retrieved 0 file names -18-08-2023 13:31:43 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: cleaner started -18-08-2023 13:31:43 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1080103072068495127","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390703132} of type:kafka_pipe_start -18-08-2023 13:31:43 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz -18-08-2023 13:31:43 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:31:43 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127, filePath:TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz -18-08-2023 13:31:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 -18-08-2023 13:31:44 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127, filePath:TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz -18-08-2023 13:31:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz -18-08-2023 13:31:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz] -18-08-2023 13:31:44 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1080103072068495127/0/0_0_1692390703840.json.gz] -18-08-2023 13:31:44 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0/insertFiles?requestId=62226d3c-a83a-4898-80f6-9e8c153b2e77&showSkippedFiles=false -18-08-2023 13:31:44 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:31:44 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:31:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:31:46 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:51 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:31:56 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:01 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:06 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:11 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: cleaner terminated -18-08-2023 13:32:15 pool-22-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:32:15 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:32:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:32:15 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:15 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1080103072068495127","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":937,"average_commit_lag_file_count":1,"start_time":1692390703131,"end_time":1692390735535,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:32:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0: service closed -18-08-2023 13:32:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 -18-08-2023 13:32:15 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 is:0, names:[] -18-08-2023 13:32:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1080103072068495127 dropped -18-08-2023 13:32:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1080103072068495127_0 dropped -18-08-2023 13:32:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 -18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:16 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8516722832153005043 -18-08-2023 13:32:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 -18-08-2023 13:32:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:16 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:32:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:32:16 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:32:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:32:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, existing:[] -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 - service started -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8516722832153005043, topicPartition: test-0 -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:32:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:32:16 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8516722832153005043. -18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 retrieved 0 file names -18-08-2023 13:32:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043. -18-08-2023 13:32:17 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 doesn't exist -18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 retrieved 0 file names -18-08-2023 13:32:17 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: cleaner started -18-08-2023 13:32:17 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8516722832153005043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390736805} of type:kafka_pipe_start -18-08-2023 13:32:17 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz -18-08-2023 13:32:17 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 190 bytes, offset 1 - 1 -18-08-2023 13:32:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043, filePath:TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz -18-08-2023 13:32:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 -18-08-2023 13:32:18 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043, filePath:TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz -18-08-2023 13:32:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz -18-08-2023 13:32:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz] -18-08-2023 13:32:18 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8516722832153005043/0/1_1_1692390737861.json.gz] -18-08-2023 13:32:18 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0/insertFiles?requestId=5bcf3187-a900-43bb-adc0-c396d49c2b9e&showSkippedFiles=false -18-08-2023 13:32:19 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:32:19 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:32:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:21 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:26 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:31 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:36 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:41 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:46 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:49 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: cleaner terminated -18-08-2023 13:32:50 pool-25-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:32:50 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:32:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:32:50 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:32:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8516722832153005043","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":190,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1196,"average_commit_lag_file_count":1,"start_time":1692390736804,"end_time":1692390770337,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:32:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0: service closed -18-08-2023 13:32:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 -18-08-2023 13:32:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 is:0, names:[] -18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8516722832153005043 dropped -18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8516722832153005043_0 dropped -18-08-2023 13:32:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 -18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2406829002217351794 -18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 -18-08-2023 13:32:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:51 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:32:51 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:32:51 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:32:51 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:32:51 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:51 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, existing:[] -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 - service started -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2406829002217351794, topicPartition: test-0 -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_2406829002217351794. -18-08-2023 13:32:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 retrieved 0 file names -18-08-2023 13:32:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794. -18-08-2023 13:32:51 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 doesn't exist -18-08-2023 13:32:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:32:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 retrieved 0 file names -18-08-2023 13:32:52 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2406829002217351794","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390771389} of type:kafka_pipe_start -18-08-2023 13:32:52 pool-28-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: cleaner started -18-08-2023 13:32:52 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz -18-08-2023 13:32:52 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 13:32:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794, filePath:TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz -18-08-2023 13:32:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 -18-08-2023 13:32:52 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794, filePath:TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz -18-08-2023 13:32:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz -18-08-2023 13:32:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz] -18-08-2023 13:32:52 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_2406829002217351794/0/0_0_1692390772094.json.gz] -18-08-2023 13:32:52 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0/insertFiles?requestId=1f106c8b-e19e-4dc9-b159-315d34768c1d&showSkippedFiles=false -18-08-2023 13:32:53 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:32:53 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:32:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:32:56 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:01 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:06 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:11 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:16 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:21 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: cleaner terminated -18-08-2023 13:33:23 pool-28-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:33:23 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:33:23 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:33:23 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:23 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_2406829002217351794","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":920,"average_commit_lag_file_count":1,"start_time":1692390771388,"end_time":1692390803907,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:33:23 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0: service closed -18-08-2023 13:33:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 -18-08-2023 13:33:23 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 is:0, names:[] -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2406829002217351794 dropped -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2406829002217351794_0 dropped -18-08-2023 13:33:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 -18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7584709008059395876 -18-08-2023 13:33:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 -18-08-2023 13:33:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:24 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:33:24 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:33:24 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:33:24 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 13:33:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:24 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, existing:[] -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 - service started -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7584709008059395876, topicPartition: test-0 -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:33:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:33:24 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_7584709008059395876. -18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 retrieved 0 file names -18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876. -18-08-2023 13:33:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 doesn't exist -18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 retrieved 0 file names -18-08-2023 13:33:25 pool-31-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: cleaner started -18-08-2023 13:33:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7584709008059395876","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692390804775} of type:kafka_pipe_start -18-08-2023 13:33:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz -18-08-2023 13:33:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 178 bytes, offset 1 - 1 -18-08-2023 13:33:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876, filePath:TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz -18-08-2023 13:33:25 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 -18-08-2023 13:33:25 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876, filePath:TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz -18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz -18-08-2023 13:33:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz] -18-08-2023 13:33:25 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_7584709008059395876/0/1_1_1692390805441.json.gz] -18-08-2023 13:33:25 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0/insertFiles?requestId=b773c5a1-ba13-4936-96f5-77f33e3aeac4&showSkippedFiles=false -18-08-2023 13:33:26 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 20:33:26 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 13:33:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:29 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:34 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:39 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:49 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:54 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:56 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: cleaner terminated -18-08-2023 13:33:57 pool-31-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 13:33:57 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:33:57 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 13:33:57 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:33:57 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_7584709008059395876","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":178,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1000,"average_commit_lag_file_count":1,"start_time":1692390804774,"end_time":1692390837339,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 13:33:57 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0: service closed -18-08-2023 13:33:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 -18-08-2023 13:33:57 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 is:0, names:[] -18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7584709008059395876 dropped -18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7584709008059395876_0 dropped -18-08-2023 13:33:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:57 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:33:57 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:33:58 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 13:33:58 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:33:58 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:33:58 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:58 Thread-16 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:33:58 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:33:58 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:33:58 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:33:58 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:33:58 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:33:58 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:33:58 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1063988221374408804 doesn't exist -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1063988221374408804. -18-08-2023 13:33:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1063988221374408804 -18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1063988221374408804 -18-08-2023 13:33:58 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1063988221374408804, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_0","table":"kafka_connector_test_table_1063988221374408804"} -18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1063988221374408804, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:33:58 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804 -18-08-2023 13:33:58 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:33:58 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset:null -18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1063988221374408804, topicPartition: test-0 -18-08-2023 13:33:58 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -18-08-2023 13:33:58 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -18-08-2023 13:33:58 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 13:33:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:33:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1046, currentDiffMillis=1050 -18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:33:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 -18-08-2023 13:33:59 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1f098399[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec] -18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, idx=0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. -18-08-2023 13:33:59 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1105: footer length = 755 -18-08-2023 13:33:59 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 755 => 243 2 0 0 -18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1113, encryptedCompressedSize=1120, bdecVersion=THREE -18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, size=1120 -18-08-2023 13:33:59 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, size=1120, timeInMillis=204 -18-08-2023 13:33:59 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec, idx=0 -18-08-2023 13:33:59 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 13:33:59 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 13:33:59 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec","md5":"4e1696ba5678fcfc54c3c6cfb2e17112","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","chunk_start_offset":0,"chunk_length":1113,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"f52d2bc6308888e944c18a30ac6ce963","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370837370,"first_insert_time_in_ms":1692390838783,"last_insert_time_in_ms":1692390838783,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390839246,"build_duration_ms":260,"upload_duration_ms":202},"bdec_version":3}],"role":"testrole_kafka","request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_2"} -18-08-2023 13:33:59 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/33/rzlt4n_CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_64_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 13:34:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 13:34:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:34:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:34:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:03 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:34:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:34:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 13:34:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 13:34:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 -18-08-2023 13:34:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:08 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:34:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:34:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:34:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 13:34:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:34:12 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:13 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:34:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1027 -18-08-2023 13:34:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 13:34:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 13:34:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:34:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:18 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 13:34:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 13:34:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 13:34:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 13:34:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:23 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:34:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:34:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 13:34:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 13:34:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:34:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:28 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:34:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1055, currentDiffMillis=1055 -18-08-2023 13:34:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0 -18-08-2023 13:34:29 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:34:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, offset:0 -18-08-2023 13:34:29 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 13:34:29 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"CoDLM2CiGBBfb1TE78N4iCOWAAtzGSQXVSqutMYOztrAAcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1063988221374408804","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:34:29 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -18-08-2023 13:34:29 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 13:34:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=407, currentDiffMillis=407 -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=23, mean=0.3032728552534317, stddev=2.064925961469176, p50=0.0, p75=0.0, p95=0.0, p98=3.0, p99=13.0, p999=23.0 -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.454872103818902, rate_unit=events/second -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.24924890737557, rate_unit=events/second -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=260.722042, max=260.722042, mean=260.722042, stddev=0.0, p50=260.722042, p75=260.722042, p95=260.722042, p98=260.722042, p99=260.722042, p999=260.722042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147083486471098, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=648.610417, max=648.610417, mean=648.610417, stddev=0.0, p50=648.610417, p75=648.610417, p95=648.610417, p98=648.610417, p99=648.610417, p999=648.610417, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147054438606101, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=181.051959, max=181.051959, mean=181.051959, stddev=0.0, p50=181.051959, p75=181.051959, p95=181.051959, p98=181.051959, p99=181.051959, p999=181.051959, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03147037746475505, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:34:29 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=202.078459, max=202.078459, mean=202.078459, stddev=0.0, p50=202.078459, p75=202.078459, p95=202.078459, p98=202.078459, p99=202.078459, p999=202.078459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031470203817609614, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:34:29 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:34:29 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:34:29 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 13:34:29 Thread-16 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:30 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:34:30 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 13:34:30 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:34:30 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:30 Thread-18 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:34:30 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:34:30 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:34:30 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:34:30 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:34:30 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:34:30 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:34:30 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3694719056936488009 doesn't exist -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3694719056936488009. -18-08-2023 13:34:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3694719056936488009 -18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_3694719056936488009 -18-08-2023 13:34:30 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3694719056936488009, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_0","table":"kafka_connector_test_table_3694719056936488009"} -18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3694719056936488009, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:34:30 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009 -18-08-2023 13:34:30 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:30 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset:null -18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3694719056936488009, topicPartition: test-0 -18-08-2023 13:34:30 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:34:30 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:34:30 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0,currentBufferSizeInBytes:229, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1} -18-08-2023 13:34:30 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 229 bytes, offset 1 - 1 -18-08-2023 13:34:30 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=229, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 13:34:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:34:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1047 -18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 -18-08-2023 13:34:31 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@71d66860[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, rowCount=1, bufferSize=83.5 in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, idx=0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [34, 123, 92, 34, 110, 97, 109, 101, 92, 34, 58, 92, 34, 116, 101, 115, 116, 92, 34, 125, 34] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(21 bytes) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 25 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 49 to byteArray of 49 bytes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 49 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 72 to byteArray of 72 bytes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 177: end column -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 72B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 31B raw, 49B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 177: end block -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 177: column indexes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 357: offset indexes -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 380: bloom filters -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 380: end -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1154: footer length = 774 -18-08-2023 13:34:31 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 774 => 6 3 0 0 -18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009, rowCount=1, startOffset=0, estimatedUncompressedSize=83.5, paddedChunkLength=1162, encryptedCompressedSize=1168, bdecVersion=THREE -18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, size=1168 -18-08-2023 13:34:31 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, size=1168, timeInMillis=286 -18-08-2023 13:34:31 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec, idx=0 -18-08-2023 13:34:31 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 13:34:31 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 13:34:31 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec","md5":"2b688fb4ea514dd03c46115d1cbe3c77","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","chunk_start_offset":0,"chunk_length":1162,"chunk_length_uncompressed":83,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"bd1e6dd7bd9625d6d138caa1d75c0c74","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370836294,"first_insert_time_in_ms":1692390870947,"last_insert_time_in_ms":1692390870947,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390871367,"build_duration_ms":7,"upload_duration_ms":285},"bdec_version":3}],"role":"testrole_kafka","request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_2"} -18-08-2023 13:34:31 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/34/rzlt5j_sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_74_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 13:34:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 13:34:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 -18-08-2023 13:34:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1051, currentDiffMillis=1051 -18-08-2023 13:34:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:35 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:34:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 13:34:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:34:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 13:34:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1040 -18-08-2023 13:34:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:40 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 -18-08-2023 13:34:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 13:34:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 13:34:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -18-08-2023 13:34:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 -18-08-2023 13:34:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:45 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1018, currentDiffMillis=1019 -18-08-2023 13:34:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1032, currentDiffMillis=1033 -18-08-2023 13:34:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 13:34:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 13:34:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 -18-08-2023 13:34:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:50 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 13:34:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 13:34:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 -18-08-2023 13:34:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:34:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1047, currentDiffMillis=1048 -18-08-2023 13:34:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:55 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:34:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -18-08-2023 13:34:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 13:34:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 13:34:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:34:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -18-08-2023 13:34:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:35:00 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1032 -18-08-2023 13:35:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:35:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 13:35:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:35:01 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0 -18-08-2023 13:35:01 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:01 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, offset:1 -18-08-2023 13:35:01 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 13:35:01 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 13:35:01 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"sXfFgEJkXGoMTB5Cdl8hi5kK9RAAevZFXOmanJW4nHK0CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3694719056936488009","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:02 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 13:35:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=598, currentDiffMillis=598 -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1168, max=1168, mean=1168.0, stddev=0.0, p50=1168.0, p75=1168.0, p95=1168.0, p98=1168.0, p99=1168.0, p999=1168.0 -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=10, mean=0.12375958740937146, stddev=0.9474530713527652, p50=0.0, p75=0.0, p95=0.0, p98=1.0, p99=7.0, p999=10.0 -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=83, m1_rate=10.943394461327369, m5_rate=15.272737282846768, m15_rate=16.14523432013138, mean_rate=2.611643738283248, rate_unit=events/second -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1168, m1_rate=153.99861121482368, m5_rate=214.9223752574099, m15_rate=227.20040585437897, mean_rate=36.75162256801035, rate_unit=events/second -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.548916, max=7.548916, mean=7.548916, stddev=0.0, p50=7.548916, p75=7.548916, p95=7.548916, p98=7.548916, p99=7.548916, p999=7.548916, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146518957110048, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=449.222917, max=449.222917, mean=449.222917, stddev=0.0, p50=449.222917, p75=449.222917, p95=449.222917, p98=449.222917, p99=449.222917, p999=449.222917, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146496874809927, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=154.766375, max=154.766375, mean=154.766375, stddev=0.0, p50=154.766375, p75=154.766375, p95=154.766375, p98=154.766375, p99=154.766375, p999=154.766375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146477119491828, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=285.631042, max=285.631042, mean=285.631042, stddev=0.0, p50=285.631042, p75=285.631042, p95=285.631042, p98=285.631042, p99=285.631042, p999=285.631042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03146453490876679, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:35:02 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 13:35:02 Thread-18 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:02 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:35:02 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:35:02 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:02 Thread-20 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:35:02 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:35:02 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:35:02 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:35:02 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:35:02 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_2, total available processors=10 -18-08-2023 13:35:02 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_2, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:35:02 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:35:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:35:02 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:35:02 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_303740236573408996 doesn't exist -18-08-2023 13:35:02 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_303740236573408996. -18-08-2023 13:35:02 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_303740236573408996 -18-08-2023 13:35:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_303740236573408996 -18-08-2023 13:35:02 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_303740236573408996, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:02 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_0","table":"kafka_connector_test_table_303740236573408996"} -18-08-2023 13:35:03 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_303740236573408996, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:03 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_303740236573408996 -18-08-2023 13:35:03 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:03 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset:null -18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:35:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_303740236573408996, topicPartition: test-0 -18-08-2023 13:35:03 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -18-08-2023 13:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -18-08-2023 13:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 13:35:03 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 -18-08-2023 13:35:03 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_2, blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@14324c4[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, idx=0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1103: footer length = 753 -18-08-2023 13:35:03 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 753 => 241 2 0 0 -18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1111, encryptedCompressedSize=1120, bdecVersion=THREE -18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, size=1120 -18-08-2023 13:35:03 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, size=1120, timeInMillis=206 -18-08-2023 13:35:03 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec, idx=0 -18-08-2023 13:35:03 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_2, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 13:35:03 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -18-08-2023 13:35:03 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec","md5":"80778765d8082a2441ab0b595be2db4a","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","chunk_start_offset":0,"chunk_length":1111,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"066a278cb6ecd399b6a10a550f4ef02e","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370838602,"first_insert_time_in_ms":1692390903266,"last_insert_time_in_ms":1692390903266,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390903690,"build_duration_ms":7,"upload_duration_ms":206},"bdec_version":3}],"role":"testrole_kafka","request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_2"} -18-08-2023 13:35:04 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/35/rzlt6f_EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_82_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_2, executionCount=0 -18-08-2023 13:35:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:35:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -18-08-2023 13:35:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:35:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:07 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 13:35:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:35:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1026 -18-08-2023 13:35:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1048, currentDiffMillis=1049 -18-08-2023 13:35:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 13:35:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:12 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:13 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:35:13 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1049, currentDiffMillis=1050 -18-08-2023 13:35:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 13:35:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1026 -18-08-2023 13:35:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 13:35:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:17 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 13:35:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1031 -18-08-2023 13:35:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 13:35:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:35:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 13:35:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:22 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 13:35:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:35:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 13:35:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -18-08-2023 13:35:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 13:35:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:27 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1041 -18-08-2023 13:35:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 13:35:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:35:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:35:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:32 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 13:35:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=false, isNeedFlush=false, timeDiffMillis=1026, currentDiffMillis=1027 -18-08-2023 13:35:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:33 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0 -18-08-2023 13:35:33 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, offset:0 -18-08-2023 13:35:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_303740236573408996.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 13:35:34 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"EBBw3GXQnu9ba07vkZuRUZIR8XmNtAFIzMdfAR1zrnG4CC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_303740236573408996","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:34 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 13:35:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_2, isForce=true, isNeedFlush=false, timeDiffMillis=380, currentDiffMillis=380 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=305, min=0, max=5, mean=0.01961205603582848, stddev=0.2667514544940201, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=0.0, p999=5.0 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.4684269534666727, rate_unit=events/second -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=35.44391070583461, rate_unit=events/second -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.464542, max=7.464542, mean=7.464542, stddev=0.0, p50=7.464542, p75=7.464542, p95=7.464542, p98=7.464542, p99=7.464542, p999=7.464542, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164611908801335, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=369.784375, max=369.784375, mean=369.784375, stddev=0.0, p50=369.784375, p75=369.784375, p95=369.784375, p98=369.784375, p99=369.784375, p999=369.784375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164592133863106, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=154.738667, max=154.738667, mean=154.738667, stddev=0.0, p50=154.738667, p75=154.738667, p95=154.738667, p98=154.738667, p99=154.738667, p999=154.738667, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03164574358071113, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=206.072875, max=206.072875, mean=206.072875, stddev=0.0, p50=206.072875, p75=206.072875, p95=206.072875, p98=206.072875, p99=206.072875, p999=206.072875, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031645555266590394, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:35:34 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:34 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_2 -18-08-2023 13:35:34 Thread-20 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:35:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:35:34 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:35:34 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:34 Thread-22 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:35:34 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:35:34 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:35:34 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:35:34 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:35:34 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:35:34 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_3, total available processors=10 -18-08-2023 13:35:34 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_3, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:35:34 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:35:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:35:34 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:35:34 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:35:35 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1456398297857472324 doesn't exist -18-08-2023 13:35:35 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1456398297857472324. -18-08-2023 13:35:35 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1456398297857472324 -18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1456398297857472324 -18-08-2023 13:35:35 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1456398297857472324, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:35:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_0","table":"kafka_connector_test_table_1456398297857472324"} -18-08-2023 13:35:35 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1456398297857472324, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:35:35 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324 -18-08-2023 13:35:35 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:35 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset:null -18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:35:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1456398297857472324, topicPartition: test-0 -18-08-2023 13:35:35 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:35:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 13:35:35 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0,currentBufferSizeInBytes:217, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1} -18-08-2023 13:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 217 bytes, offset 1 - 1 -18-08-2023 13:35:35 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=217, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 13:35:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:35:35 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1045 -18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:35 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 -18-08-2023 13:35:35 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_3, blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@1aecae47[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, rowCount=1, bufferSize=77.5 in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec] -18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 13:35:35 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, idx=0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 110, 97, 109, 101, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(15 bytes) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 19 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 19 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 43 to byteArray of 43 bytes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 43 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 66 to byteArray of 66 bytes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 171: end column -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 66B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 25B raw, 43B comp, 1 pages, encodings: [PLAIN] -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 171: end block -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 171: column indexes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 339: offset indexes -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 362: bloom filters -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 362: end -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1112: footer length = 750 -18-08-2023 13:35:35 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 750 => 238 2 0 0 -18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324, rowCount=1, startOffset=0, estimatedUncompressedSize=77.5, paddedChunkLength=1120, encryptedCompressedSize=1136, bdecVersion=THREE -18-08-2023 13:35:35 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, size=1136 -18-08-2023 13:35:36 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, size=1136, timeInMillis=212 -18-08-2023 13:35:36 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec, idx=0 -18-08-2023 13:35:36 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_3, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 13:35:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -18-08-2023 13:35:36 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec","md5":"b2c1fc2deecfd9f1067a1271ff16acce","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","chunk_start_offset":0,"chunk_length":1120,"chunk_length_uncompressed":77,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"6e99914afdcd9f57663286e65d3a4347","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370841702,"first_insert_time_in_ms":1692390935590,"last_insert_time_in_ms":1692390935590,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692390935887,"build_duration_ms":7,"upload_duration_ms":211},"bdec_version":3}],"role":"testrole_kafka","request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_2"} -18-08-2023 13:35:36 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/20/35/rzlt7b_gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_90_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_3, executionCount=0 -18-08-2023 13:35:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1028, currentDiffMillis=1028 -18-08-2023 13:35:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 13:35:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1042 -18-08-2023 13:35:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:39 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 13:35:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 13:35:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 13:35:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:35:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1023, currentDiffMillis=1023 -18-08-2023 13:35:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:44 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 13:35:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 13:35:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 -18-08-2023 13:35:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:48 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 13:35:48 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:35:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:49 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 13:35:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 13:35:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1019, currentDiffMillis=1020 -18-08-2023 13:35:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 13:35:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 13:35:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:54 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:35:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 13:35:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 13:35:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 13:35:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 13:35:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:35:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:35:59 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:36:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 13:36:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1040 -18-08-2023 13:36:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1027, currentDiffMillis=1027 -18-08-2023 13:36:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1034 -18-08-2023 13:36:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:04 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:36:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1054, currentDiffMillis=1055 -18-08-2023 13:36:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 13:36:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:06 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0 -18-08-2023 13:36:06 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:36:06 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, offset:1 -18-08-2023 13:36:06 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 13:36:06 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"gm8I2GyaLwsOhroN2iWA013gUHFrCk3YYu0p83AAj8tcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1456398297857472324","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:36:06 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 13:36:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 13:36:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_3, isForce=true, isNeedFlush=false, timeDiffMillis=616, currentDiffMillis=616 -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1136, max=1136, mean=1136.0, stddev=0.0, p50=1136.0, p75=1136.0, p95=1136.0, p98=1136.0, p99=1136.0, p999=1136.0 -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=307, min=0, max=4, mean=0.038359723140007185, stddev=0.3263378480837562, p50=0.0, p75=0.0, p95=0.0, p98=0.0, p99=1.0, p999=4.0 -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=77, m1_rate=10.152305705086835, m5_rate=14.16868398529158, m15_rate=14.978108947591764, mean_rate=2.4180631262543066, rate_unit=events/second -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1136, m1_rate=149.77947118154083, m5_rate=209.03409100378224, m15_rate=220.97573720083432, mean_rate=35.67416513032052, rate_unit=events/second -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=7.022, max=7.022, mean=7.022, stddev=0.0, p50=7.022, p75=7.022, p95=7.022, p98=7.022, p99=7.022, p999=7.022, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0314031454534754, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=377.704209, max=377.704209, mean=377.704209, stddev=0.0, p50=377.704209, p75=377.704209, p95=377.704209, p98=377.704209, p99=377.704209, p999=377.704209, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03140296950719061, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=157.849375, max=157.849375, mean=157.849375, stddev=0.0, p50=157.849375, p75=157.849375, p95=157.849375, p98=157.849375, p99=157.849375, p999=157.849375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03140286830423283, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:36:06 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=211.400459, max=211.400459, mean=211.400459, stddev=0.0, p50=211.400459, p75=211.400459, p95=211.400459, p98=211.400459, p99=211.400459, p999=211.400459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031402734846510606, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 13:36:06 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:36:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 13:36:06 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_3 -18-08-2023 13:36:06 Thread-22 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 13:36:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:07 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:36:07 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:07 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:07 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:07 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:36:07 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:36:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:36:07 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:07 Thread-24 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 13:36:07 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:36:07 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:36:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:36:07 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_4 -18-08-2023 13:36:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:36:07 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_4, total available processors=10 -18-08-2023 13:36:07 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_4, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:36:07 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_4 -18-08-2023 13:36:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:36:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:36:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:36:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_1976254417471840300 doesn't exist -18-08-2023 13:36:07 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_1976254417471840300. -18-08-2023 13:36:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1976254417471840300 -18-08-2023 13:36:07 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_1976254417471840300 -18-08-2023 13:36:07 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1976254417471840300, client=KC_CLIENT_TEST_CONNECTOR_4 -18-08-2023 13:36:07 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"J70P5icEwn0OWxlmpzOmSOVQSAA7WNiBmGZjnMm6gINkCC_1003_0","table":"kafka_connector_test_table_1976254417471840300"} -18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_1976254417471840300, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_4 -18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300 -18-08-2023 13:36:08 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:36:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 -18-08-2023 13:36:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"J70P5icEwn0OWxlmpzOmSOVQSAA7WNiBmGZjnMm6gINkCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:36:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, offset:null -18-08-2023 13:36:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1976254417471840300, topicPartition: test-0 -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_4, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 13:36:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 -18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:08 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 13:36:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 13:36:08 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:08 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:08 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 13:36:08 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 13:36:08 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 13:36:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 13:36:08 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:08 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 13:36:08 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 13:36:08 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 13:36:08 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_5 -18-08-2023 13:36:08 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 13:36:08 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_5, total available processors=10 -18-08-2023 13:36:08 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_5, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 13:36:08 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_5 -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 13:36:08 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4342516535815138887 doesn't exist -18-08-2023 13:36:08 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4342516535815138887. -18-08-2023 13:36:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4342516535815138887 -18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4342516535815138887 -18-08-2023 13:36:09 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4342516535815138887, client=KC_CLIENT_TEST_CONNECTOR_5 -18-08-2023 13:36:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"d4D0b97qlayku4mqQP6JtTo3IQe586Y3NeSf5RJ1QBBUCC_1003_0","table":"kafka_connector_test_table_4342516535815138887"} -18-08-2023 13:36:09 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4342516535815138887, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_5 -18-08-2023 13:36:09 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887 -18-08-2023 13:36:09 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 13:36:09 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0 -18-08-2023 13:36:09 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"d4D0b97qlayku4mqQP6JtTo3IQe586Y3NeSf5RJ1QBBUCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, offset:null -18-08-2023 13:36:09 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4342516535815138887.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 13:36:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4342516535815138887, topicPartition: test-0 -18-08-2023 13:36:09 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 13:36:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 13:36:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_4, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 13:36:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_1976254417471840300.TEST_0 -18-08-2023 14:38:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:38:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 -18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 -18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5224312851784726463 -18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 -18-08-2023 14:38:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 -18-08-2023 14:38:15 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:38:15 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:38:15 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:38:15 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:38:15 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:38:15 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 -18-08-2023 14:38:15 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0, existing:[] -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 - service started -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_5224312851784726463, topicPartition: test-0 -18-08-2023 14:38:15 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5224312851784726463 dropped -18-08-2023 14:38:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5224312851784726463_0 dropped -18-08-2023 14:38:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:38:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 -18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7794421424625032973 -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 -18-08-2023 14:38:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 -18-08-2023 14:38:16 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:38:16 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:38:16 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:38:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 -18-08-2023 14:38:16 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0, existing:[] -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 - service started -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_7794421424625032973, topicPartition: test-0 -18-08-2023 14:38:16 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7794421424625032973 dropped -18-08-2023 14:38:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7794421424625032973_0 dropped -18-08-2023 14:38:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:39:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 -18-08-2023 14:39:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 -18-08-2023 14:39:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2657774367744592480 -18-08-2023 14:39:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:39:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 -18-08-2023 14:39:31 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 -18-08-2023 14:39:31 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:39:31 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:39:31 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:39:31 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:39:31 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:39:31 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 -18-08-2023 14:39:31 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0, existing:[] -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 - service started -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_2657774367744592480, topicPartition: test-0 -18-08-2023 14:39:31 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:39:40 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 14:39:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2657774367744592480 dropped -18-08-2023 14:39:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2657774367744592480_0 dropped -18-08-2023 14:39:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 14:39:54 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:54 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 -18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 -18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3366560785811907597 -18-08-2023 14:39:55 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3366560785811907597 -18-08-2023 14:39:55 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 -18-08-2023 14:39:55 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:39:55 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:39:55 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:39:55 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:39:55 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:39:55 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 -18-08-2023 14:39:55 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0, existing:[] -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3366560785811907597_0 - service started -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3366560785811907597, topicPartition: test-0 -18-08-2023 14:39:55 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:40:00 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 14:40:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:40:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 -18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 -18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6286529608848783160 -18-08-2023 14:40:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6286529608848783160 -18-08-2023 14:40:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 -18-08-2023 14:40:10 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:40:10 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:40:10 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:40:10 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:40:10 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:40:10 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 -18-08-2023 14:40:10 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0, existing:[] -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6286529608848783160_0 - service started -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6286529608848783160, topicPartition: test-0 -18-08-2023 14:40:10 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:41:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:41:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1644140466561349223 -18-08-2023 14:41:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1644140466561349223_0 -18-08-2023 14:42:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:42:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:42:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:42:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2821288052890949006 -18-08-2023 14:42:34 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2821288052890949006_0 -18-08-2023 14:42:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:42:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:42:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 -18-08-2023 14:42:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:42 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:42:42 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:42:42 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:42:42 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:42:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3121063502165281348 -18-08-2023 14:42:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:42:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 -18-08-2023 14:42:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:43 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:42:43 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:42:43 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:42:43 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:42:43 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:42:43 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:43 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, existing:[] -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 - service started -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3121063502165281348, topicPartition: test-0 -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:42:43 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 14:42:43 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_3121063502165281348. -18-08-2023 14:42:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 retrieved 0 file names -18-08-2023 14:42:43 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348. -18-08-2023 14:42:43 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 doesn't exist -18-08-2023 14:42:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:42:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 retrieved 0 file names -18-08-2023 14:42:44 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: cleaner started -18-08-2023 14:42:44 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3121063502165281348","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692394963406} of type:kafka_pipe_start -18-08-2023 14:42:44 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz -18-08-2023 14:42:44 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 -18-08-2023 14:42:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348, filePath:TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz -18-08-2023 14:42:44 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 -18-08-2023 14:42:44 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348, filePath:TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz -18-08-2023 14:42:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz -18-08-2023 14:42:44 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz] -18-08-2023 14:42:44 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_3121063502165281348/0/1_1_1692394964190.json.gz] -18-08-2023 14:42:44 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0/insertFiles?requestId=d5b59727-3ec7-4059-a245-b1b21238cc8c&showSkippedFiles=false -18-08-2023 14:42:45 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:42:45 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:42:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:42:48 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:42:53 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:42:58 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:03 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:08 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:13 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: cleaner terminated -18-08-2023 14:43:16 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:43:16 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:43:16 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:43:16 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:16 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_3121063502165281348","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1329,"average_commit_lag_file_count":1,"start_time":1692394963367,"end_time":1692394996326,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:43:16 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0: service closed -18-08-2023 14:43:16 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 -18-08-2023 14:43:16 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 is:0, names:[] -18-08-2023 14:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3121063502165281348 dropped -18-08-2023 14:43:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3121063502165281348_0 dropped -18-08-2023 14:43:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 -18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:17 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:17 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:17 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:17 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6727268077568408945 -18-08-2023 14:43:17 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 -18-08-2023 14:43:17 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:17 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:43:17 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 14:43:17 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:43:17 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:43:17 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:43:17 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:17 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, existing:[] -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 - service started -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_6727268077568408945, topicPartition: test-0 -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:17 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_6727268077568408945. -18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 retrieved 0 file names -18-08-2023 14:43:18 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945. -18-08-2023 14:43:18 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 doesn't exist -18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 retrieved 0 file names -18-08-2023 14:43:18 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: cleaner started -18-08-2023 14:43:18 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6727268077568408945","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692394997676} of type:kafka_pipe_start -18-08-2023 14:43:18 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz -18-08-2023 14:43:18 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 14:43:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945, filePath:TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz -18-08-2023 14:43:18 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 -18-08-2023 14:43:19 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945, filePath:TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz -18-08-2023 14:43:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz -18-08-2023 14:43:19 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz] -18-08-2023 14:43:19 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_6727268077568408945/0/0_0_1692394998414.json.gz] -18-08-2023 14:43:19 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0/insertFiles?requestId=72341438-dc1f-4590-8d68-6493546bd830&showSkippedFiles=false -18-08-2023 14:43:19 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:43:19 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:43:19 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:22 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:27 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:32 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:37 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:42 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:47 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:50 pool-4-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:43:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: cleaner terminated -18-08-2023 14:43:50 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:43:50 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:43:50 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:43:50 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_6727268077568408945","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1308,"average_commit_lag_file_count":1,"start_time":1692394997668,"end_time":1692395030429,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:43:50 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0: service closed -18-08-2023 14:43:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 -18-08-2023 14:43:50 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 is:0, names:[] -18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6727268077568408945 dropped -18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6727268077568408945_0 dropped -18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:50 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:50 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 -18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:51 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:51 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:51 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:43:51 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_152605127061635629 -18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 -18-08-2023 14:43:51 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:51 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:43:51 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:43:51 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:43:51 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:43:51 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:51 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, existing:[] -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 - service started -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_152605127061635629, topicPartition: test-0 -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:43:51 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 14:43:51 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_152605127061635629. -18-08-2023 14:43:51 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 retrieved 0 file names -18-08-2023 14:43:51 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629. -18-08-2023 14:43:51 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 doesn't exist -18-08-2023 14:43:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:43:52 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 retrieved 0 file names -18-08-2023 14:43:52 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: cleaner started -18-08-2023 14:43:52 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_152605127061635629","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395031532} of type:kafka_pipe_start -18-08-2023 14:43:52 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz -18-08-2023 14:43:52 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 -18-08-2023 14:43:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629, filePath:TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz -18-08-2023 14:43:52 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 -18-08-2023 14:43:52 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629, filePath:TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz -18-08-2023 14:43:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz -18-08-2023 14:43:52 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz] -18-08-2023 14:43:52 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_152605127061635629/0/1_1_1692395032272.json.gz] -18-08-2023 14:43:52 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0/insertFiles?requestId=f23cec22-e7d4-436c-8074-976667a7d8ed&showSkippedFiles=false -18-08-2023 14:43:53 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:43:53 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:43:53 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:43:56 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:01 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:06 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:16 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:21 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: cleaner terminated -18-08-2023 14:44:24 pool-7-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:44:24 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:44:24 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:44:24 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:24 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_152605127061635629","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1131,"average_commit_lag_file_count":1,"start_time":1692395031480,"end_time":1692395064218,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:44:24 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0: service closed -18-08-2023 14:44:24 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 -18-08-2023 14:44:24 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 is:0, names:[] -18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_152605127061635629 dropped -18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_152605127061635629_0 dropped -18-08-2023 14:44:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:44:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 -18-08-2023 14:44:24 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:24 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:24 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:24 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:24 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:44:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_445993476782435834 -18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:44:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 -18-08-2023 14:44:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:25 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:44:25 Thread-4 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:44:25 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:44:25 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:44:25 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:25 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, existing:[] -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 - service started -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_445993476782435834, topicPartition: test-0 -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_445993476782435834. -18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 retrieved 0 file names -18-08-2023 14:44:25 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834. -18-08-2023 14:44:25 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 doesn't exist -18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 retrieved 0 file names -18-08-2023 14:44:25 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_445993476782435834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395065101} of type:kafka_pipe_start -18-08-2023 14:44:25 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: cleaner started -18-08-2023 14:44:25 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz -18-08-2023 14:44:25 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 14:44:26 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834, filePath:TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz -18-08-2023 14:44:26 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 -18-08-2023 14:44:26 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834, filePath:TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz -18-08-2023 14:44:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz -18-08-2023 14:44:26 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz] -18-08-2023 14:44:26 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_445993476782435834/0/0_0_1692395065946.json.gz] -18-08-2023 14:44:26 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0/insertFiles?requestId=1b6fd0b8-81a9-4bd8-81b5-96c425eeed88&showSkippedFiles=false -18-08-2023 14:44:27 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:44:27 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:44:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:30 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:35 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:40 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:45 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:50 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:55 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:57 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: cleaner terminated -18-08-2023 14:44:58 pool-10-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:44:58 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:44:58 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:44:58 Thread-4 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:44:58 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_445993476782435834","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1672,"average_commit_lag_file_count":1,"start_time":1692395065100,"end_time":1692395098295,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:44:58 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0: service closed -18-08-2023 14:44:58 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 -18-08-2023 14:44:58 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 is:0, names:[] -18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_445993476782435834 dropped -18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_445993476782435834_0 dropped -18-08-2023 14:44:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:58 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:44:58 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:44:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 -18-08-2023 14:44:58 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:44:58 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:58 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:58 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:44:58 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1398553495658597523 -18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:44:59 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 -18-08-2023 14:44:59 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:44:59 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:44:59 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:44:59 Thread-6 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:44:59 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:44:59 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:44:59 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, existing:[] -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 - service started -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1398553495658597523, topicPartition: test-0 -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:44:59 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 14:44:59 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1398553495658597523. -18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 retrieved 0 file names -18-08-2023 14:44:59 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523. -18-08-2023 14:44:59 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 doesn't exist -18-08-2023 14:44:59 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:45:00 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 retrieved 0 file names -18-08-2023 14:45:00 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: cleaner started -18-08-2023 14:45:00 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1398553495658597523","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395099302} of type:kafka_pipe_start -18-08-2023 14:45:00 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz -18-08-2023 14:45:00 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 -18-08-2023 14:45:00 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523, filePath:TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz -18-08-2023 14:45:00 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 -18-08-2023 14:45:00 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523, filePath:TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz -18-08-2023 14:45:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz -18-08-2023 14:45:00 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz] -18-08-2023 14:45:00 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1398553495658597523/0/1_1_1692395100067.json.gz] -18-08-2023 14:45:00 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0/insertFiles?requestId=cdef9147-086d-4024-b7bc-150893b30e49&showSkippedFiles=false -18-08-2023 14:45:01 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:45:01 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:45:01 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:04 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:09 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:14 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:19 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:24 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:29 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: cleaner terminated -18-08-2023 14:45:32 pool-13-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:45:32 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:45:32 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:45:32 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1398553495658597523","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1292,"average_commit_lag_file_count":1,"start_time":1692395099300,"end_time":1692395132217,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:45:32 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0: service closed -18-08-2023 14:45:32 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 -18-08-2023 14:45:32 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 is:0, names:[] -18-08-2023 14:45:32 Thread-6 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1398553495658597523 dropped -18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1398553495658597523_0 dropped -18-08-2023 14:45:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:45:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 -18-08-2023 14:45:32 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:32 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:45:32 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:45:32 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:45:32 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:45:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8503004783070881087 -18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:45:33 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 -18-08-2023 14:45:33 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:33 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:45:33 Thread-8 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:45:33 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:45:33 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:45:33 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:33 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, existing:[] -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 - service started -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_8503004783070881087, topicPartition: test-0 -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_8503004783070881087. -18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 retrieved 0 file names -18-08-2023 14:45:33 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087. -18-08-2023 14:45:33 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 doesn't exist -18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:45:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 retrieved 0 file names -18-08-2023 14:45:33 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: cleaner started -18-08-2023 14:45:33 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8503004783070881087","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395133115} of type:kafka_pipe_start -18-08-2023 14:45:33 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz -18-08-2023 14:45:33 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 180 bytes, offset 0 - 0 -18-08-2023 14:45:33 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087, filePath:TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz -18-08-2023 14:45:34 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 -18-08-2023 14:45:34 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087, filePath:TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz -18-08-2023 14:45:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz -18-08-2023 14:45:34 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz] -18-08-2023 14:45:34 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_8503004783070881087/0/0_0_1692395133861.json.gz] -18-08-2023 14:45:34 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0/insertFiles?requestId=c6f95f5c-fcb7-4337-8eba-756935129d41&showSkippedFiles=false -18-08-2023 14:45:35 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:45:35 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:45:35 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:45:38 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:43 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:48 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:53 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:45:58 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:03 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:05 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:05 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: cleaner terminated -18-08-2023 14:46:05 pool-16-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:46:06 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:46:06 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:46:06 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_8503004783070881087","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0","processed-offset":0,"flushed-offset":0,"committed-offset":0,"purged-offset":-1,"record_number":1,"byte_number":180,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1422,"average_commit_lag_file_count":1,"start_time":1692395133115,"end_time":1692395166003,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:46:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0: service closed -18-08-2023 14:46:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 -18-08-2023 14:46:06 Thread-8 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:06 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 is:0, names:[] -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8503004783070881087 dropped -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8503004783070881087_0 dropped -18-08-2023 14:46:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:06 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 -18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:06 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:06 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:06 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:06 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1712531299485417574 -18-08-2023 14:46:06 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 -18-08-2023 14:46:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkServiceV1 created -18-08-2023 14:46:06 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] set number of record limitation to 1 -18-08-2023 14:46:06 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 -18-08-2023 14:46:06 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:06 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:46:06 Thread-10 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:46:07 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:46:07 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: SFKafkaConnector/2.0.0 provider/UNKNOWN -18-08-2023 14:46:07 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the pipe connector for pipe testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:07 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, existing:[] -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Registered 14 metrics for pipeName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 - service started -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_1712531299485417574, topicPartition: test-0 -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:46:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 14:46:07 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] init pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing table kafka_connector_test_table_1712531299485417574. -18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 retrieved 0 file names -18-08-2023 14:46:07 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Using existing stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574. -18-08-2023 14:46:07 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 doesn't exist -18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create pipe: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] list stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 retrieved 0 file names -18-08-2023 14:46:07 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1712531299485417574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0","is_reuse_table":true,"is_reuse_stage":true,"is_reuse_pipe":false,"file_count_restart":0,"file_count_reprocess_purge":0,"start_time":1692395167010} of type:kafka_pipe_start -18-08-2023 14:46:07 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: cleaner started -18-08-2023 14:46:07 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz -18-08-2023 14:46:07 main DEBUG SnowflakeSinkServiceV1:70 - [SF_KAFKA_CONNECTOR] flush buffer: 1 records, 172 bytes, offset 1 - 1 -18-08-2023 14:46:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Query credential(Refreshing Credentials) for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574, filePath:TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz -18-08-2023 14:46:07 main DEBUG SnowflakeInternalStage:70 - [SF_KAFKA_CONNECTOR] Caching credential successful for stage:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 -18-08-2023 14:46:08 main INFO SnowflakeInternalStage:46 - [SF_KAFKA_CONNECTOR] uploadWithoutConnection successful for stageName:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574, filePath:TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz -18-08-2023 14:46:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, flush pipe: TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz -18-08-2023 14:46:08 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0, ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz] -18-08-2023 14:46:08 main DEBUG SnowflakeIngestionServiceV1:70 - [SF_KAFKA_CONNECTOR] ingest files: [TEST_CONNECTOR/kafka_connector_test_table_1712531299485417574/0/1_1_1692395167737.json.gz] -18-08-2023 14:46:08 main INFO RequestBuilder:675 - Created Insert Request : https://sfctest0.snowflakecomputing.com:443/v1/data/pipes/testdb_kafka.kafka_test.SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0/insertFiles?requestId=5ac8ab08-9436-4d32-b745-3d4e89be2d31&showSkippedFiles=false -18-08-2023 14:46:08 main INFO SimpleIngestManager:568 - Attempting to unmarshall insert response - HttpResponseProxy{HTTP/1.1 200 OK [Content-Type: application/json, Date: Fri, 18 Aug 2023 21:46:08 GMT, Expect-CT: enforce, max-age=3600, Strict-Transport-Security: max-age=31536000, Vary: Accept-Encoding, User-Agent, X-Content-Type-Options: nosniff, X-Country: United States, X-Frame-Options: deny, X-XSS-Protection: 1; mode=block, Content-Length: 88, Connection: keep-alive] ResponseEntityProxy{[Content-Type: application/json,Content-Length: 88,Chunked: false]}}, with clientInfo - null -18-08-2023 14:46:08 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:12 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:17 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:22 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:27 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:32 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:37 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: cleaner terminated -18-08-2023 14:46:39 pool-19-thread-1 INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] Cleaner terminated by an interrupt: -sleep interrupted -18-08-2023 14:46:39 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:46:39 main INFO SnowflakeIngestionServiceV1:46 - [SF_KAFKA_CONNECTOR] IngestService Closed -18-08-2023 14:46:39 Thread-10 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_CONNECTOR","task_id":"-1","snowflake.ingestion.method":0,"table_name":"kafka_connector_test_table_1712531299485417574","stage_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574","pipe_name":"SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0","processed-offset":1,"flushed-offset":1,"committed-offset":1,"purged-offset":-1,"record_number":1,"byte_number":172,"file-count-on-stage":1,"file-count-on-ingestion":1,"file-count-purged":0,"file_count_table_stage_ingest_fail":0,"file-count-table-stage-broken-record":0,"cleaner_restart_count":0,"memory_usage":0,"average_kafka_lag":0,"average_kafka_lag_record_count":0,"average_ingestion_lag":0,"average_ingestion_lag_file_count":0,"average_commit_lag":1037,"average_commit_lag_file_count":1,"start_time":1692395167009,"end_time":1692395199721,"is_pipe_closing":true} of type:kafka_pipe_usage -18-08-2023 14:46:39 main INFO SnowflakeSinkServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0: service closed -18-08-2023 14:46:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Unregistering all metrics for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 -18-08-2023 14:46:39 main DEBUG MetricsJmxReporter:70 - [SF_KAFKA_CONNECTOR] Metric registry size for pipe:SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 is:0, names:[] -18-08-2023 14:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1712531299485417574 dropped -18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1712531299485417574_0 dropped -18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:40 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:46:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:46:40 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:46:40 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 14:46:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 14:46:40 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:40 Thread-12 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:46:40 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:46:40 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:46:40 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 14:46:40 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:46:40 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 14:46:40 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_0, total available processors=10 -18-08-2023 14:46:40 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_0, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 14:46:40 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:46:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 14:46:40 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 14:46:40 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:46:41 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_3234142018577459629 doesn't exist -18-08-2023 14:46:41 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_3234142018577459629. -18-08-2023 14:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3234142018577459629 -18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_3234142018577459629 -18-08-2023 14:46:41 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3234142018577459629, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:46:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_0","table":"kafka_connector_test_table_3234142018577459629"} -18-08-2023 14:46:41 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_3234142018577459629, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:46:41 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629 -18-08-2023 14:46:41 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset:null -18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 14:46:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_3234142018577459629, topicPartition: test-0 -18-08-2023 14:46:41 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0,currentBufferSizeInBytes:219, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0} -18-08-2023 14:46:41 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 219 bytes, offset 0 - 0 -18-08-2023 14:46:41 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=219, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 14:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:46:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 -18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 -18-08-2023 14:46:41 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_0, blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@71d00f62[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec] -18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 14:46:41 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, idx=0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, rowCount=1, bufferSize=78.5 in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:44 - Trying to load the custom-built native-hadoop library... -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:53 - Failed to load native-hadoop with error: java.lang.UnsatisfiedLinkError: no hadoop in java.library.path: [/Users/rcheng/Library/Java/Extensions, /Library/Java/Extensions, /Network/Library/Java/Extensions, /System/Library/Java/Extensions, /usr/lib/java, .] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG NativeCodeLoader:54 - java.library.path=/Users/rcheng/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:. -18-08-2023 14:46:41 ingest-build-upload-thread-0 WARN NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -18-08-2023 14:46:41 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 48, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 115, 99, 104, 101, 109, 97, 95, 105, 100, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(68 bytes) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 125] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(2 bytes) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 6 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 30 to byteArray of 30 bytes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 30 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 53 to byteArray of 53 bytes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 72 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 72 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 84 to byteArray of 84 bytes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 84 bytes to out -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 109 to byteArray of 109 bytes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 113: end column -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 109B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 78B raw, 84B comp, 1 pages, encodings: [PLAIN] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 113: write data pages -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 113: write data pages content -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 166: end column -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 53B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 12B raw, 30B comp, 1 pages, encodings: [PLAIN] -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 166: end block -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 166: column indexes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 328: offset indexes -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 350: bloom filters -18-08-2023 14:46:41 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 350: end -18-08-2023 14:46:42 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1102: footer length = 752 -18-08-2023 14:46:42 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 752 => 240 2 0 0 -18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629, rowCount=1, startOffset=0, estimatedUncompressedSize=78.5, paddedChunkLength=1110, encryptedCompressedSize=1120, bdecVersion=THREE -18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, size=1120 -18-08-2023 14:46:42 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, size=1120, timeInMillis=227 -18-08-2023 14:46:42 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec, idx=0 -18-08-2023 14:46:42 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_0, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 14:46:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 14:46:42 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec","md5":"2a4347b46d9af90d096205cb144abfbf","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","chunk_start_offset":0,"chunk_length":1110,"chunk_length_uncompressed":78,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"0"}],"chunk_md5":"763fcfa1d59c43f1250c1b36ae390cc6","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370973898,"first_insert_time_in_ms":1692395201726,"last_insert_time_in_ms":1692395201726,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692395201762,"build_duration_ms":360,"upload_duration_ms":217},"bdec_version":3}],"role":"testrole_kafka","request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_2"} -18-08-2023 14:46:42 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/21/46/rzlwht_nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_52_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_0, executionCount=0 -18-08-2023 14:46:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 14:46:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1035 -18-08-2023 14:46:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 14:46:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:45 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:45 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1042 -18-08-2023 14:46:45 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:46 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1025, currentDiffMillis=1025 -18-08-2023 14:46:46 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:47 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1041, currentDiffMillis=1041 -18-08-2023 14:46:47 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:49 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1029 -18-08-2023 14:46:49 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:50 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 14:46:50 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:50 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:51 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 14:46:51 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:52 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 14:46:52 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:53 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1046 -18-08-2023 14:46:53 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:54 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1038 -18-08-2023 14:46:54 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:55 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 -18-08-2023 14:46:55 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:55 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:46:56 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 14:46:56 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:57 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 14:46:57 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:58 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 14:46:58 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:46:59 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 14:46:59 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:00 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 14:47:00 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:00 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:01 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1043 -18-08-2023 14:47:01 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:02 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 14:47:02 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:03 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1038 -18-08-2023 14:47:03 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:04 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 14:47:04 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:05 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:05 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1036 -18-08-2023 14:47:05 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:06 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 14:47:06 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:07 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 14:47:07 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:08 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 14:47:08 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:09 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 14:47:09 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:10 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:10 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 -18-08-2023 14:47:10 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:11 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -18-08-2023 14:47:11 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:12 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0 -18-08-2023 14:47:12 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:47:12 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, offset:0 -18-08-2023 14:47:12 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 14:47:12 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"nRRGRSRuewtOBVoEakN7JEwYnkGg3Xs6Y3FTx6BimkICC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_3234142018577459629","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:47:12 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=0, persistedRowSequencer=1, persistedOffsetToken=0 -18-08-2023 14:47:12 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 14:47:12 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_0, isForce=true, isNeedFlush=false, timeDiffMillis=805, currentDiffMillis=805 -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1120, max=1120, mean=1120.0, stddev=0.0, p50=1120.0, p75=1120.0, p95=1120.0, p98=1120.0, p99=1120.0, p999=1120.0 -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=309, min=0, max=18, mean=0.23795522761607765, stddev=1.7584974094692836, p50=0.0, p75=0.0, p95=0.0, p98=3.0, p99=16.0, p999=18.0 -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=78, m1_rate=10.284153831126925, m5_rate=14.352692868217444, m15_rate=15.172629843015034, mean_rate=2.422997937709851, rate_unit=events/second -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1120, m1_rate=147.66990116489944, m5_rate=206.08994887696844, m15_rate=217.86340287406202, mean_rate=34.79121362272457, rate_unit=events/second -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=360.579375, max=360.579375, mean=360.579375, stddev=0.0, p50=360.579375, p75=360.579375, p95=360.579375, p98=360.579375, p99=360.579375, p999=360.579375, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031061614020123102, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=759.624459, max=759.624459, mean=759.624459, stddev=0.0, p50=759.624459, p75=759.624459, p95=759.624459, p98=759.624459, p99=759.624459, p999=759.624459, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031061192720059357, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=169.259042, max=169.259042, mean=169.259042, stddev=0.0, p50=169.259042, p75=169.259042, p95=169.259042, p98=169.259042, p99=169.259042, p999=169.259042, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03106096715971506, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:12 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=217.7195, max=217.7195, mean=217.7195, stddev=0.0, p50=217.7195, p75=217.7195, p95=217.7195, p98=217.7195, p99=217.7195, p999=217.7195, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031060744497920734, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:12 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:47:12 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:47:12 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:12 Thread-12 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:12 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 14:47:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 14:47:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:47:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:47:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 14:47:13 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 14:47:13 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is disabled, creating a new streaming client -18-08-2023 14:47:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 14:47:13 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:13 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 14:47:13 Thread-14 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 14:47:13 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 14:47:13 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 14:47:13 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 14:47:13 main INFO FlushService:58 - [SF_INGEST] Create 30 threads for build/upload blobs for client=KC_CLIENT_TEST_CONNECTOR_1, total available processors=10 -18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Client created, name=KC_CLIENT_TEST_CONNECTOR_1, account=sfctest0. isTestMode=false, parameters=ParameterProvider{parameterMap={enable_snowpipe_streaming_jmx_metrics=false, blob_format_version=THREE, io_time_cpu_ratio=2, enable_parquet_internal_buffering=false, streaming_ingest_client_sdk_insert_throttle_threshold_in_percentage=10, max_channel_size_in_bytes=32000000, streaming_ingest_client_sdk_insert_throttle_threshold_in_bytes=209715200, max_memory_limit_in_bytes=-1, blob_upload_max_retry_count=24, max_chunk_size_in_bytes=128000000, streaming_ingest_client_sdk_buffer_flush_interval_in_millis=1000, streaming_ingest_client_sdk_buffer_flush_check_interval_in_millis=100, streaming_ingest_client_sdk_insert_throttle_interval_in_millis=1000}} -18-08-2023 14:47:13 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully initialized Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2 created -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Set number of records for buffer threshold to 1 -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] record number is limited to 1 -18-08-2023 14:47:13 main DEBUG SnowflakeConnectionServiceV1:70 - [SF_KAFKA_CONNECTOR] table kafka_connector_test_table_4949091797102522534 doesn't exist -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Creating new table kafka_connector_test_table_4949091797102522534. -18-08-2023 14:47:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4949091797102522534 -18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:test_0 for table name:kafka_connector_test_table_4949091797102522534 -18-08-2023 14:47:13 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Open channel request start, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4949091797102522534, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/open/, payload={"schema":"kafka_test","write_mode":"CLOUD_STORAGE","database":"testdb_kafka","role":"testrole_kafka","channel":"test_0","request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_0","table":"kafka_connector_test_table_4949091797102522534"} -18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Open channel request succeeded, channel=test_0, table=testdb_kafka.kafka_test.kafka_connector_test_table_4949091797102522534, clientSequencer=0, rowSequencer=0, client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:13 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel=TEST_0 created for table=KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534 -18-08-2023 14:47:13 main DEBUG SnowflakeStreamingIngestChannelInternal:70 - [SF_INGEST] Setup schema for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, schema=[{physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_METADATA, length=null, scale=null, type=VARIANT}, {physical_type=LOB, nullable=true, logical_type=VARIANT, precision=null, byte_length=null, name=RECORD_CONTENT, length=null, scale=null, type=VARIANT}] -18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:13 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_1","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset:null -18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] create new task in com.snowflake.kafka.connector.internal.SnowflakeSinkService - table: kafka_connector_test_table_4949091797102522534, topicPartition: test-0 -18-08-2023 14:47:13 main INFO SnowflakeSinkServiceFactory$SnowflakeSinkServiceBuilder:46 - [SF_KAFKA_CONNECTOR] com.snowflake.kafka.connector.internal.SnowflakeSinkService created -18-08-2023 14:47:13 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Record value is null for topic test, partition 0 and offset 0 -18-08-2023 14:47:13 main DEBUG RecordService:70 - [SF_KAFKA_CONNECTOR] Null valued record from topic 'test', partition 0 and offset 0 was skipped. -18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0,currentBufferSizeInBytes:211, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=20000000, bufferRecordCountThreshold=1} -18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1} -18-08-2023 14:47:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 211 bytes, offset 1 - 1 -18-08-2023 14:47:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=211, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 14:47:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:14 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1040 -18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush about to take lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:14 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Buffer flush released lock on channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 -18-08-2023 14:47:14 ingest-flush-thread INFO FlushService:58 - [SF_INGEST] buildAndUpload task added for client=KC_CLIENT_TEST_CONNECTOR_1, blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, buildUploadWorkers stats=java.util.concurrent.ThreadPoolExecutor@3e6d4889[Running, pool size = 1, active threads = 1, queued tasks = 0, completed tasks = 0] -18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop outer for uploading blobs=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec] -18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start loop inner for uploading blobs, size=1, idx=0 -18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Start waiting on uploading blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, idx=0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Start building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFlusher:70 - [SF_INGEST] Parquet Flusher: Finish building channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, rowCount=1, bufferSize=74.5 in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, enableParquetMemoryOptimization=false -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:423 - 0: start -18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO CodecPool:153 - Got brand-new compressor [.gz] -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RunLengthBitPackingHybridEncoder:119 - Encoding: RunLengthBitPackingHybridEncoder with bithWidth: 1 initialCapacity 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE START > -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_METADATA, 0) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_METADATA -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 116, 111, 112, 105, 99, 34, 58, 34, 116, 101, 115, 116, 34, 44, 34, 111, 102, 102, 115, 101, 116, 34, 58, 49, 44, 34, 112, 97, 114, 116, 105, 116, 105, 111, 110, 34, 58, 48, 44, 34, 107, 101, 121, 34, 58, 34, 116, 101, 115, 116, 34, 125] -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(54 bytes) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={}}: [RECORD_METADATA] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_METADATA, 0) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - startField(RECORD_CONTENT, 1) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ValidatingRecordConsumer:167 - validate [BINARY, INT96, FIXED_LEN_BYTE_ARRAY] for RECORD_CONTENT -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - [123, 34, 105, 110, 116, 49, 54, 34, 58, 49, 50, 125] -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - addBinary(12 bytes) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - r: 0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0}}: [RECORD_CONTENT] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - endField(RECORD_CONTENT, 1) -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - < MESSAGE END > -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG MessageColumnIO:283 - 0, VisitedIndex{visitedIndexes={0, 1}}: [] r:0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG RecordConsumerLoggingWrapper:69 - -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG InternalParquetRecordWriter:177 - Flushing mem columnStore to file. allocated memory: 2048 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:442 - 4: start block -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 16 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 16 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 40 to byteArray of 40 bytes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 23 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 40 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 63 to byteArray of 63 bytes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:191 - used 0 slabs, adding new slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG PlainValuesWriter:118 - writing a buffer of size 58 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 0 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 6 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:318 - { -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 4 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 2 => 2 0 0 0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 2 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:320 - } -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 58 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 76 to byteArray of 76 bytes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 25 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput$SequenceBytesIn:317 - write 76 bytes to out -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesInput:227 - converted 101 to byteArray of 101 bytes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 64 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG CapacityByteArrayOutputStream:285 - initial slab of size 1024 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 4: write data pages -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 4: write data pages content -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 105: end column -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 101B for [RECORD_METADATA] optional binary RECORD_METADATA (STRING) = 1: 1 values, 64B raw, 76B comp, 1 pages, encodings: [PLAIN] -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:897 - 105: write data pages -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:901 - 105: write data pages content -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:925 - 168: end column -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ColumnChunkPageWriteStore:347 - written 63B for [RECORD_CONTENT] optional binary RECORD_CONTENT (STRING) = 2: 1 values, 22B raw, 40B comp, 1 pages, encodings: [PLAIN] -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:962 - 168: end block -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1212 - 168: column indexes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1253 - 330: offset indexes -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1286 - 352: bloom filters -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1201 - 352: end -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG ParquetFileWriter:1339 - 1088: footer length = 736 -18-08-2023 14:47:14 ingest-build-upload-thread-0 DEBUG BytesUtils:162 - write le int: 736 => 224 2 0 0 -18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO BlobBuilder:58 - [SF_INGEST] Finish building chunk in blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, table=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534, rowCount=1, startOffset=0, estimatedUncompressedSize=74.5, paddedChunkLength=1096, encryptedCompressedSize=1104, bdecVersion=THREE -18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Start uploading file=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, size=1104 -18-08-2023 14:47:14 ingest-build-upload-thread-0 INFO FlushService:58 - [SF_INGEST] Finish uploading file=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, size=1104, timeInMillis=233 -18-08-2023 14:47:14 ingest-register-thread DEBUG RegisterService:70 - [SF_INGEST] Finish waiting on uploading blob=2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec, idx=0 -18-08-2023 14:47:14 ingest-register-thread INFO RegisterService:58 - [SF_INGEST] Start registering blobs in client=KC_CLIENT_TEST_CONNECTOR_1, totalBlobListSize=1, currentBlobListSize=1, idx=1 -18-08-2023 14:47:14 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request preparing for blob=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 14:47:14 ingest-register-thread DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/write/blobs/, payload={"blobs":[{"path":"2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec","md5":"5679568e03cc27abebdbafacebb985c5","chunks":[{"schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","chunk_start_offset":0,"chunk_length":1096,"chunk_length_uncompressed":74,"channels":[{"channel_name":"TEST_0","client_sequencer":0,"row_sequencer":1,"offset_token":"1"}],"chunk_md5":"4e6ca2c9f706aa30ac9b711e1d22e0e7","eps":{"rows":1,"columns":{"RECORD_METADATA":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0},"RECORD_CONTENT":{"minStrValue":null,"maxStrValue":null,"collation":null,"minStrNonCollated":null,"maxStrNonCollated":null,"minIntValue":0,"maxIntValue":0,"minRealValue":0.0,"maxRealValue":0.0,"distinctValues":-1,"nullCount":0,"maxLength":0}}},"encryption_key_id":13920370970226,"first_insert_time_in_ms":1692395233832,"last_insert_time_in_ms":1692395233832,"database":"TESTDB_KAFKA"}],"blob_stats":{"flush_start_ms":1692395234255,"build_duration_ms":28,"upload_duration_ms":232},"bdec_version":3}],"role":"testrole_kafka","request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_2"} -18-08-2023 14:47:14 ingest-register-thread INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Register blob request returned for blob=[2023/8/18/21/47/rzlwiq_BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_63_0.bdec], client=KC_CLIENT_TEST_CONNECTOR_1, executionCount=0 -18-08-2023 14:47:15 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1029, currentDiffMillis=1030 -18-08-2023 14:47:15 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:16 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1045, currentDiffMillis=1045 -18-08-2023 14:47:16 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:17 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1043, currentDiffMillis=1044 -18-08-2023 14:47:17 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:18 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:18 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1036 -18-08-2023 14:47:18 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:19 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 14:47:19 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:20 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 14:47:20 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:21 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1035, currentDiffMillis=1035 -18-08-2023 14:47:21 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:22 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 14:47:22 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:23 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:23 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1038, currentDiffMillis=1039 -18-08-2023 14:47:23 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:24 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 14:47:24 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:25 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 14:47:25 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:26 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 14:47:26 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:27 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 14:47:27 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:28 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:28 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 14:47:28 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:29 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 14:47:29 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:30 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1031, currentDiffMillis=1031 -18-08-2023 14:47:30 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:31 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1043 -18-08-2023 14:47:31 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:32 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 14:47:32 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:33 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:33 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1033, currentDiffMillis=1033 -18-08-2023 14:47:33 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:34 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1034 -18-08-2023 14:47:34 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:36 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1030, currentDiffMillis=1030 -18-08-2023 14:47:36 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:37 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1042, currentDiffMillis=1042 -18-08-2023 14:47:37 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:38 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1034, currentDiffMillis=1035 -18-08-2023 14:47:38 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:38 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:39 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 14:47:39 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:40 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1039, currentDiffMillis=1039 -18-08-2023 14:47:40 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:41 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1040, currentDiffMillis=1040 -18-08-2023 14:47:41 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:42 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1036, currentDiffMillis=1037 -18-08-2023 14:47:42 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:43 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 14:47:43 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1037, currentDiffMillis=1037 -18-08-2023 14:47:43 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=false, isNeedFlush=false, timeDiffMillis=1044, currentDiffMillis=1044 -18-08-2023 14:47:44 ingest-flush-thread DEBUG AbstractRowBuffer:70 - [SF_INGEST] Start get data for channel=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:44 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0 -18-08-2023 14:47:44 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_3","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:47:44 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, offset:1 -18-08-2023 14:47:44 main INFO SnowflakeSinkServiceV2:46 - [SF_KAFKA_CONNECTOR] Closing partition channel:test_0 -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestChannelInternal:58 - [SF_INGEST] Channel is marked as closed, name=TESTDB_KAFKA.KAFKA_TEST.KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534.TEST_0, channel sequencer=0, row sequencer=1 -18-08-2023 14:47:44 ForkJoinPool.commonPool-worker-19 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/channels/status/, payload={"request_id":"BKc8HiC3gKRDu32kaGGEIZ7EZJgU5noEUNT17DSiYJcCC_1003_4","channels":[{"database":"TESTDB_KAFKA","schema":"KAFKA_TEST","table":"KAFKA_CONNECTOR_TEST_TABLE_4949091797102522534","channel_name":"TEST_0","client_sequencer":0}],"role":"testrole_kafka"} -18-08-2023 14:47:44 ForkJoinPool.commonPool-worker-19 INFO SnowflakeStreamingIngestClientInternal:58 - [SF_INGEST] Get channel status name=TEST_0, status=0, clientSequencer=0, rowSequencer=1, offsetToken=1, persistedRowSequencer=1, persistedOffsetToken=1 -18-08-2023 14:47:44 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 14:47:44 ingest-flush-thread TRACE FlushService:28 - [SF_INGEST] Submit forced or ad-hoc flush task on client=KC_CLIENT_TEST_CONNECTOR_1, isForce=true, isNeedFlush=false, timeDiffMillis=449, currentDiffMillis=453 -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.row.count.histogram, count=1, min=1, max=1, mean=1.0, stddev=0.0, p50=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0 -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=blob.size.histogram, count=1, min=1104, max=1104, mean=1104.0, stddev=0.0, p50=1104.0, p75=1104.0, p95=1104.0, p98=1104.0, p99=1104.0, p999=1104.0 -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=HISTOGRAM, name=cpu.usage.histogram, count=306, min=0, max=4, mean=0.09143120584939526, stddev=0.5150330101608892, p50=0.0, p75=0.0, p95=0.0, p98=2.0, p99=4.0, p999=4.0 -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.input, count=74, m1_rate=9.75676132696657, m5_rate=13.616657336513985, m15_rate=14.394546261321954, mean_rate=2.335870957716532, rate_unit=events/second -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=METER, name=throughput.upload, count=1104, m1_rate=145.560331148258, m5_rate=203.14580675015458, m15_rate=214.7510685472897, mean_rate=34.84848828603301, rate_unit=events/second -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.build, count=1, min=28.986625, max=28.986625, mean=28.986625, stddev=0.0, p50=28.986625, p75=28.986625, p95=28.986625, p98=28.986625, p99=28.986625, p999=28.986625, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031564586104728655, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.flush, count=1, min=408.814917, max=408.814917, mean=408.814917, stddev=0.0, p50=408.814917, p75=408.814917, p95=408.814917, p98=408.814917, p99=408.814917, p999=408.814917, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.031561944450599025, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.register, count=1, min=137.507583, max=137.507583, mean=137.507583, stddev=0.0, p50=137.507583, p75=137.507583, p95=137.507583, p98=137.507583, p99=137.507583, p999=137.507583, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.0315616024409613, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:44 main INFO SnowflakeStreamingIngestClientInternal:95 - type=TIMER, name=latency.upload, count=1, min=232.043417, max=232.043417, mean=232.043417, stddev=0.0, p50=232.043417, p75=232.043417, p95=232.043417, p98=232.043417, p99=232.043417, p999=232.043417, m1_rate=0.1318481260400888, m5_rate=0.18400888292586465, m15_rate=0.1945208954232697, mean_rate=0.03156135253663886, rate_unit=events/second, duration_unit=milliseconds -18-08-2023 14:47:44 main DEBUG SnowflakeStreamingIngestClientInternal:70 - [SF_INGEST] Unregistering all metrics for client=KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:44 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 14:47:44 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:KC_CLIENT_TEST_CONNECTOR_1 -18-08-2023 14:47:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 14:47:44 Thread-14 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 1; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 1; max: 100] - -18-08-2023 15:35:00 main DEBUG CodecFactory:71 - Snappy was not available -java.lang.NoClassDefFoundError: org/xerial/snappy/Snappy - at org.apache.avro.file.SnappyCodec$Option.(SnappyCodec.java:35) - at org.apache.avro.file.CodecFactory.snappyCodec(CodecFactory.java:69) - at org.apache.avro.file.CodecFactory.(CodecFactory.java:140) - at org.apache.avro.file.DataFileStream.resolveCodec(DataFileStream.java:160) - at org.apache.avro.file.DataFileStream.initialize(DataFileStream.java:144) - at org.apache.avro.file.DataFileReader.(DataFileReader.java:143) - at org.apache.avro.file.DataFileReader.(DataFileReader.java:130) - at com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry.toConnectData(SnowflakeAvroConverterWithoutSchemaRegistry.java:49) - at com.snowflake.kafka.connector.records.ProcessRecordTest.getAvroWithoutRegistryValue(ProcessRecordTest.java:184) - at com.snowflake.kafka.connector.records.ProcessRecordTest.data(ProcessRecordTest.java:69) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.runners.Parameterized$RunnersFactory.allParameters(Parameterized.java:424) - at org.junit.runners.Parameterized$RunnersFactory.(Parameterized.java:375) - at org.junit.runners.Parameterized$RunnersFactory.(Parameterized.java:360) - at org.junit.runners.Parameterized.(Parameterized.java:303) - at java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) - at java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) - at java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:490) - at org.junit.internal.builders.AnnotatedBuilder.buildRunner(AnnotatedBuilder.java:104) - at org.junit.internal.builders.AnnotatedBuilder.runnerForClass(AnnotatedBuilder.java:86) - at org.junit.runners.model.RunnerBuilder.safeRunnerForClass(RunnerBuilder.java:70) - at org.junit.internal.builders.AllDefaultPossibilitiesBuilder.runnerForClass(AllDefaultPossibilitiesBuilder.java:37) - at org.junit.runners.model.RunnerBuilder.safeRunnerForClass(RunnerBuilder.java:70) - at org.junit.runners.model.RunnerBuilder.runners(RunnerBuilder.java:125) - at org.junit.runners.model.RunnerBuilder.runners(RunnerBuilder.java:111) - at com.intellij.junit4.JUnit46ClassesRequestBuilder.collectWrappedRunners(JUnit46ClassesRequestBuilder.java:77) - at com.intellij.junit4.JUnit46ClassesRequestBuilder.getClassesRequest(JUnit46ClassesRequestBuilder.java:39) - at com.intellij.junit4.JUnit4TestRunnerUtil.buildRequest(JUnit4TestRunnerUtil.java:71) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:47) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.lang.ClassNotFoundException: org.xerial.snappy.Snappy - at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:581) - at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) - at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522) - ... 41 more -18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] invalid JDBC_LOG_DIR /dummy_dir_not_exist defaulting to /var/folders/hg/t5ql_h_514n9p951h854x1nw0000gn/T/ -18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] jdbc tracing directory = /usr -18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] jdbc tracing directory = /var/folders/hg/t5ql_h_514n9p951h854x1nw0000gn/T/ -18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: adsadas -18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] table name @123 should have at least 2 characters, start with _a-zA-Z, and only contains _$a-zA-z0-9 -18-08-2023 15:35:00 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] Current Snowflake Kafka Connector Version: 2.0.0 -18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: .*_cat, big_.*_cat -18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: .*_cat, .*_dog.*_cat -18-08-2023 15:35:00 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: 12321 -18-08-2023 15:35:00 main ERROR SnowflakeSinkTask_STATIC:94 - [SF_KAFKA_CONNECTOR] Invalid Input, Topic2Table Map disabled -18-08-2023 15:35:00 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:00 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:01 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:01 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:02 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:02 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:02 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:03 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:860, currentBufferedRecordCount:4, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=800, bufferRecordCountThreshold=10000000} -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=4, bufferSizeBytes=860, firstOffset=0, lastOffset=3} -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 4 records, 860 bytes, offset 0 - 3 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=4, bufferSizeBytes=860, firstOffset=0, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:03 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:03 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398108033, previousFlushTimeMs:1692398103019, bufferThresholdSeconds:5 -18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=215, firstOffset=4, lastOffset=4} -18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 215 bytes, offset 4 - 4 -18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=215, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:08 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:08 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Skip adding offset:0 to buffer for channel:TEST_0 because offsetPersistedInSnowflake:0, processedOffset:0 -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398113072, previousFlushTimeMs:1692398108058, bufferThresholdSeconds:5 -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=430, firstOffset=1, lastOffset=2} -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 430 bytes, offset 1 - 2 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=430, firstOffset=1, lastOffset=2}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:13 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:11878, currentBufferedRecordCount:2, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=10000000} -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=11878, firstOffset=0, lastOffset=1} -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 11878 bytes, offset 0 - 1 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=11878, firstOffset=0, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:13 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:13 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:1 -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398118160, previousFlushTimeMs:1692398113148, bufferThresholdSeconds:5 -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=5939, firstOffset=2, lastOffset=2} -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 5939 bytes, offset 2 - 2 -18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=5939, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:2 -18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:invalidNo -18-08-2023 15:35:18 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] The offsetToken string does not contain a parsable long:invalidNo for channel:TEST_0 -18-08-2023 15:35:18 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:18 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:19 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:19 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:20 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:0, reset kafka offset to:1 -18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:20 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:20 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:21 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:21 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Insert Row Error message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming Channel name:TEST_0 msg:Interrupted Exception -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:185, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=185, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 185 bytes, offset 0 - 0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=185, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 1 - 1 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 2 - 2 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 3 - 3 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 4 - 4 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:22 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 0 - 0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Got the desired offset:1 from Kafka, we can add this offset to buffer for channel:TEST_0 -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 1 - 1 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 2 - 2 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 3 - 3 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=3, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:231, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4} -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 231 bytes, offset 4 - 4 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=231, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:4 -18-08-2023 15:35:22 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:22 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:23 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:23 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:24 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:807, currentBufferedRecordCount:3, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=800, bufferRecordCountThreshold=10000000} -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=3, bufferSizeBytes=807, firstOffset=0, lastOffset=2} -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 3 records, 807 bytes, offset 0 - 2 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=3, bufferSizeBytes=807, firstOffset=0, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:24 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:24 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398129353, previousFlushTimeMs:1692398124341, bufferThresholdSeconds:5 -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=538, firstOffset=3, lastOffset=4} -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 538 bytes, offset 3 - 4 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=538, firstOffset=3, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:100 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:29 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:29 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:29 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:29 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:12098, currentBufferedRecordCount:2, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=5, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=10000000} -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=12098, firstOffset=0, lastOffset=1} -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 2 records, 12098 bytes, offset 0 - 1 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=2, bufferSizeBytes=12098, firstOffset=0, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:29 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:29 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:1 -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Time based flush for channel:TEST_0, CurrentTimeMs:1692398134389, previousFlushTimeMs:1692398129373, bufferThresholdSeconds:5 -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=6049, firstOffset=2, lastOffset=2} -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 6049 bytes, offset 2 - 2 -18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=6049, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:2 -18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:invalidNo -18-08-2023 15:35:34 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] The offsetToken string does not contain a parsable long:invalidNo for channel:TEST_0 -18-08-2023 15:35:34 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:34 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:35 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:35 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:36 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:0, reset kafka offset to:1 -18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:0 -18-08-2023 15:35:36 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:36 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:37 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:1, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:37 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_RETRY_POLICY] retry for getLatestCommittedOffsetToken. Retry no:2, message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [GET_OFFSET_TOKEN_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] [OFFSET_TOKEN_FALLBACK] Failed to open Channel/fetch offsetToken for channel:TEST_0, exception:net.snowflake.ingest.utils.SFException: Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Insert Row Error message:Channel INVALID_CHANNEL is invalid and might contain uncommitted rows, please consider reopening the channel to restart. -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:true, needToResetOffset:false -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main ERROR TopicPartitionChannel:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming Channel name:TEST_0 msg:Interrupted Exception -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:227, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=227, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 227 bytes, offset 0 - 0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=227, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] TopicPartitionChannel:TEST_0, offset token is NULL, will rely on Kafka to send us the correct offset instead -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 1 - 1 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 2 - 2 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 3 - 3 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 4 - 4 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] Failed Attempt to invoke the insertRows API for buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Re-opening channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Opening a channel with name:TEST_0 for table name:TEST_TABLE -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Fetching offsetToken after re-opening the channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:null -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, offset token is NULL, will use the consumer offset managed by the connector instead, consumer offset:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [RESET_PARTITION] Emptying current buffer:StreamingBuffer{numOfRecords=0, bufferSizeBytes=0, firstOffset=-1, lastOffset=-1} for Channel:TEST_0 due to reset of offsets in kafka -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_ROWS_FALLBACK] Channel:TEST_0, OffsetRecoveredFromSnowflake:-1, reset kafka offset to:0 -18-08-2023 15:35:38 main WARN TopicPartitionChannel:82 - [SF_KAFKA_CONNECTOR] [INSERT_BUFFERED_RECORDS] Failure inserting buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 0 - 0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=0, lastOffset=0}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Got the desired offset:1 from Kafka, we can add this offset to buffer for channel:TEST_0 -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 1 - 1 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=1, lastOffset=1}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 2 - 2 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=2, lastOffset=2}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 3 - 3 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=3, lastOffset=3}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Flush based on buffered bytes or buffered number of records for channel:TEST_0,currentBufferSizeInBytes:273, currentBufferedRecordCount:1, connectorBufferThresholds:StreamingBufferThreshold{bufferFlushTimeThreshold=10, bufferByteSizeThreshold=10000, bufferRecordCountThreshold=1} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Invoking insertRows API for channel:TEST_0, streamingBuffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4} -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Get rows for streaming ingest. 1 records, 273 bytes, offset 4 - 4 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Successfully called insertRows for channel:TEST_0, buffer:StreamingBuffer{numOfRecords=1, bufferSizeBytes=273, firstOffset=4, lastOffset=4}, insertResponseHasErrors:false, needToResetOffset:false -18-08-2023 15:35:38 main DEBUG TopicPartitionChannel:70 - [SF_KAFKA_CONNECTOR] Fetching last committed offset for partition channel:TEST_0 -18-08-2023 15:35:38 main INFO TopicPartitionChannel:46 - [SF_KAFKA_CONNECTOR] Fetched offsetToken for channelName:TEST_0, offset:4 -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time is 0, it should be greater than 1' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid snowflake.topic2table.map config format: $@#$#@%^$12312 -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.topic2table.map' is invalid. Error message: 'Invalid snowflake.topic2table.map config format: $@#$#@%^$12312' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'key.converter' is invalid. Error message: 'Config:key.converter has provided value:com.snowflake.kafka.connector.records.SnowflakeJsonConverter. If ingestionMethod is:snowpipe_streaming, Snowflake Custom Converters are not allowed. -Invalid Converters:[com.snowflake.kafka.connector.records.SnowflakeJsonConverter, com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry, com.snowflake.kafka.connector.records.SnowflakeAvroConverter]' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' -Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' -Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' -Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' -Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' -Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.enable.schematization' is invalid. Error message: 'Schematization is only available with snowpipe_streaming.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid for configuration errors.tolerance: String must be one of: none, all' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.role.name' is invalid. Error message: 'Config:snowflake.role.name should be present if ingestionMethod is:snowpipe_streaming' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'provider' is invalid. Error message: 'Kafka provider config error:Unsupported provider name: Something_which_is_not_supported. Supported are: unknown,self_hosted,confluent' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records should be a positive integer. Provided:adssadsa' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'buffer.size.bytes is too low at 0. It must be 1 or greater.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes should be an integer. Provided:afdsa' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid_value for configuration snowflake.ingestion.method: String must be one of: snowpipe, snowpipe_streaming' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'value.converter' is invalid. Error message: 'Config:value.converter has provided value:com.snowflake.kafka.connector.records.SnowflakeJsonConverter. If ingestionMethod is:snowpipe_streaming, Snowflake Custom Converters are not allowed. -Invalid Converters:[com.snowflake.kafka.connector.records.SnowflakeJsonConverter, com.snowflake.kafka.connector.records.SnowflakeAvroConverterWithoutSchemaRegistry, com.snowflake.kafka.connector.records.SnowflakeAvroConverter]' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jvm.proxy.port' is invalid. Error message: 'proxy host and port must be provided together' -Config value 'jvm.proxy.host' is invalid. Error message: 'proxy host and port must be provided together' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jvm.proxy.port' is invalid. Error message: 'proxy host and port must be provided together' -Config value 'jvm.proxy.host' is invalid. Error message: 'proxy host and port must be provided together' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.ingestion.method' is invalid. Error message: 'Kafka config:snowflake.ingestion.method error:Invalid value invalid for configuration errors.log.enable: String must be one of: true, false' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time should be an integer. Invalid integer was provided:fdas' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'enable.streaming.client.optimization' is invalid. Error message: 'Streaming client optimization is only available with snowpipe_streaming.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records should be a positive integer. Provided:adssadsa' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is -1, it should at least 1' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'org.apache.kafka.connect.storage.StringConverter' is invalid. Error message: 'The value converter:org.apache.kafka.connect.storage.StringConverter is not supported with schematization.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time should be an integer. Invalid integer was provided:fdas' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'behavior.on.null.values' is invalid. Error message: 'Kafka config:behavior.on.null.values error:Invalid value invalid for configuration behavior.on.null.values: String must be one of: default, ignore' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is -1, it should at least 1' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'Config buffer.flush.time is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] table name !@#@!#!@ should have at least 2 characters, start with _a-zA-Z, and only contains _$a-zA-z0-9 -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic name topic1 is duplicated -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] topic regexes cannot overlap. overlapping regexes: topic1, topic1 -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'jmx' is invalid. Error message: 'Kafka config:jmx should either be true or false' - -18-08-2023 15:35:38 main INFO Utils:46 - [SF_KAFKA_CONNECTOR] enable jvm proxy: 127.0.0.1:3128 and bypass proxy for hosts: *.snowflakecomputing.com|*.amazonaws.com -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.streaming.file.version' is invalid. Error message: 'snowflake.streaming.file.version is only available with ingestion type: snowpipe_streaming.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' -Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' -Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' -Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' -Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' -Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' -Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'snowflake.database.name' is invalid. Error message: 'snowflake.database.name cannot be empty.' -Config value 'snowflake.schema.name' is invalid. Error message: 'snowflake.schema.name cannot be empty.' -Config value 'buffer.count.records' is invalid. Error message: 'Config buffer.count.records is empty' -Config value 'snowflake.private.key' is invalid. Error message: 'snowflake.private.key cannot be empty.' -Config value 'snowflake.user.name' is invalid. Error message: 'snowflake.user.name cannot be empty.' -Config value 'name' is invalid. Error message: 'name is empty or invalid. It should match Snowflake object identifier syntax. Please see the documentation.' -Config value 'snowflake.url.name' is invalid. Error message: 'snowflake.url.name cannot be empty.' -Config value 'buffer.size.bytes' is invalid. Error message: 'Config buffer.size.bytes is empty' - -18-08-2023 15:35:38 main ERROR Utils:94 - [SF_KAFKA_CONNECTOR] Invalid config: Config value 'buffer.flush.time' is invalid. Error message: 'buffer.flush.time is 9, it should be greater than 10' - -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse JSON record -net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonParseException: Unexpected character ('a' (code 97)): Expected space separating root-level values - at [Source: (byte[])"123adsada"; line: 1, column: 5] -18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 100 - scrub.invalid.names = false - -18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse JSON record -net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonParseException: Unrecognized token 'fasfas': was expecting (JSON String, Number, Array, Object or token 'null', 'true' or 'false') - at [Source: (byte[])"fasfas"; line: 1, column: 7] -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: unknown bytes -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -null -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -null -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] Failed to parse AVRO record -[SF_KAFKA_CONNECTOR] Exception: Invalid input record -Error Code: 0010 -Detail: Input record value can't be parsed -Message: Failed to parse AVRO record -Not an Avro data file. -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] the string provided for reader.schema is no valid Avro schema: com.fasterxml.jackson.core.JsonParseException: Unexpected character (':' (code 58)): expected a valid value (JSON String, Number, Array, Object or token 'null', 'true' or 'false') - at [Source: (String)"{"name":"test_avro","type":"record","fields":[{"name":"int","type":"int"},{"name":"newfield","type":"int","default": 1},{"name":"missingfield","type"::"int"}]}"; line: 1, column: 152] -18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. -18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main DEBUG SslFactory:228 - Created SSL context with keystore null, truststore null, provider SunJSSE. -18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:35:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:35:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:35:39 main ERROR SnowflakeConverter:94 - [SF_KAFKA_CONNECTOR] failed to parse AVRO record -null -18-08-2023 15:35:39 main DEBUG InternalUtils:70 - [SF_KAFKA_CONNECTOR] converted date: 2019-07-18T23:32:38Z -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://account.snowflake.com:80 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.snowflake.com:443 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: account.snowflake.com:80 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: account.snowflake.com -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://account.snowflake.com -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.snowflake.com -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.region.aws.privatelink.snowflake.com:443 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: http://org-account.snowflake.com:80 -18-08-2023 15:35:39 main INFO SnowflakeURL:46 - [SF_KAFKA_CONNECTOR] enabling JDBC tracing -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://account.region.aws.privatelink.snowflake.com:443 -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: https://org-account.privatelink.snowflake.com:80 -18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated file name: TEST_CONNECTOR/test_topic/123/456_789_1692398139152.json.gz -18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/test_topic/123/456_key_1692398139158.gz -18-08-2023 15:35:39 main DEBUG FileNameUtils:70 - [SF_KAFKA_CONNECTOR] generated broken data file name: TEST_CONNECTOR/test_topic/123/456_value_1692398139158.gz -18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryBasicInfo:70 - [SF_KAFKA_CONNECTOR] Registering metrics for pipe:pipe, existing:[] -18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe","start_time":1692398139249,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start -18-08-2023 15:35:39 main DEBUG SnowflakeTelemetryService:70 - [SF_KAFKA_CONNECTOR] sending telemetry data: {"app_name":"TEST_APP","task_id":"1","snowflake.ingestion.method":"snowpipe_streaming","start_time":1692398139255,"kafka_version":"3.4.0","max_tasks":null,"buffer.size.bytes":"5000000","buffer.count.records":"10000","buffer.flush.time":"120","key.converter":"org.apache.kafka.connect.storage.StringConverter","value.converter":"io.confluent.connect.avro.AvroConverter","snowflake.enable.schematization":"false","enable.streaming.client.optimization":"true"} of type:kafka_start -18-08-2023 15:35:39 main DEBUG KCLogger:23 - Setting MDC context enablement to: true. MDC context is only available for Apache Kafka versions after 2.3.0 -18-08-2023 15:35:39 main DEBUG KCLogger:23 - Setting MDC context enablement to: false. MDC context is only available for Apache Kafka versions after 2.3.0 -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 15:35:39 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Failure closing Streaming client, Exception message: cant close client, cause: missing exception cause -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Closing Streaming Client... -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Successfully closed Streaming Client:TEST_CONNECTOR -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Config is overridden for snowflake.streaming.file.version -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 15:35:39 main INFO Utils:22 - [SF_INGEST] Snowflake role is not provided, the default user role will be applied. -18-08-2023 15:35:39 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 Thread-0 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 15:35:39 main INFO RequestBuilder:323 - Default user agent SnowpipeJavaSDK/2.0.2 (Mac OS X 13.5 aarch64) JAVA/11.0.16.1 -18-08-2023 15:35:39 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 15:35:39 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 15:35:39 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 15:35:39 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 15:35:39 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":null} -18-08-2023 15:35:39 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:39 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 0][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:39 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:39 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-0: Shutdown connection -18-08-2023 15:35:39 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 0][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:39 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 -18-08-2023 15:35:39 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:39 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:39 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:39 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 1][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:39 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:39 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:39 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-1: Shutdown connection -18-08-2023 15:35:39 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:39 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 1][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:39 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 -18-08-2023 15:35:40 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:40 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:40 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:40 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:40 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 2][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:40 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:40 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:40 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-2: Shutdown connection -18-08-2023 15:35:40 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:40 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 2][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:40 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 -18-08-2023 15:35:42 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:42 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:42 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:42 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:42 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 3][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:42 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:42 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:42 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-3: Shutdown connection -18-08-2023 15:35:42 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:42 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 3][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:42 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 -18-08-2023 15:35:44 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:35:44 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:35:44 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:35:46 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:46 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:46 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:46 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:46 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 4][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:46 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:46 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:46 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-4: Shutdown connection -18-08-2023 15:35:46 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:46 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 4][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:46 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 -18-08-2023 15:35:49 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:35:49 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:35:49 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:35:50 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:50 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:50 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:50 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:50 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 5][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:50 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:50 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:50 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-5: Shutdown connection -18-08-2023 15:35:50 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:50 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 5][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:50 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 -18-08-2023 15:35:54 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:35:54 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:35:54 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:35:54 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:54 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:54 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:54 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:54 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 6][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:54 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:54 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:54 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-6: Shutdown connection -18-08-2023 15:35:54 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:54 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 6][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:54 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 -18-08-2023 15:35:58 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:35:58 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:35:58 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:58 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:35:58 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 7][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:35:58 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:35:58 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:35:58 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-7: Shutdown connection -18-08-2023 15:35:58 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:35:58 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 7][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:35:58 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 -18-08-2023 15:35:59 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:35:59 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:35:59 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:02 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:02 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:02 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:02 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:02 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 8][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:02 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:02 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:02 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-8: Shutdown connection -18-08-2023 15:36:02 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:02 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 8][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:02 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 -18-08-2023 15:36:04 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:04 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:04 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:06 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:06 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:06 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:06 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:06 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 9][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:06 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:06 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:06 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-9: Shutdown connection -18-08-2023 15:36:06 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:06 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 9][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:06 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 -18-08-2023 15:36:09 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:09 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:09 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:10 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:10 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClientException(StreamingClientHandlerTest.java:57) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:10 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 10][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-10: Shutdown connection -18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 10][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ -18-08-2023 15:36:10 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 15:36:10 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient -18-08-2023 15:36:10 Thread-0 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:10 Thread-0 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:10 Thread-0 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:10 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 15:36:10 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 Thread-1 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 15:36:10 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 15:36:10 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 15:36:10 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 15:36:10 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 15:36:10 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 11][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-11: Shutdown connection -18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 11][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 -18-08-2023 15:36:10 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:10 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:10 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:10 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 12][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:10 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:10 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:10 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-12: Shutdown connection -18-08-2023 15:36:10 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:10 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 12][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:10 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 -18-08-2023 15:36:11 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:11 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:11 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:11 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:11 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 13][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:11 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:11 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:11 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-13: Shutdown connection -18-08-2023 15:36:11 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:11 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 13][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:11 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 -18-08-2023 15:36:13 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:13 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:13 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:13 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:13 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 14][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:13 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:13 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:13 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-14: Shutdown connection -18-08-2023 15:36:13 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:13 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 14][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:13 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 -18-08-2023 15:36:15 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:15 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:15 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:17 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:17 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:17 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:17 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:17 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 15][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:17 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:17 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:17 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-15: Shutdown connection -18-08-2023 15:36:17 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:17 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 15][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:17 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 -18-08-2023 15:36:20 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:20 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:20 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:21 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:21 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:21 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:21 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:21 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 16][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:21 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:21 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:21 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-16: Shutdown connection -18-08-2023 15:36:21 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:21 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 16][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:21 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 -18-08-2023 15:36:25 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:25 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:25 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:25 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:25 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:25 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:25 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:25 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 17][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:25 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:25 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:25 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-17: Shutdown connection -18-08-2023 15:36:25 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:25 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 17][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:25 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 -18-08-2023 15:36:29 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:29 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:29 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:29 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:29 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 18][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:29 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:29 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:29 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-18: Shutdown connection -18-08-2023 15:36:29 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:29 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 18][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:29 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 -18-08-2023 15:36:30 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:30 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:30 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:33 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:33 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:33 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:33 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:33 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 19][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:33 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:33 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:33 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-19: Shutdown connection -18-08-2023 15:36:33 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:33 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 19][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:33 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 -18-08-2023 15:36:35 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:35 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:35 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:37 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:37 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:37 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:37 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:37 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 20][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:37 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:37 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:37 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-20: Shutdown connection -18-08-2023 15:36:37 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:37 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 20][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:37 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 -18-08-2023 15:36:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:36:40 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:40 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:40 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:41 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:41 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandlerTest.testCreateClient(StreamingClientHandlerTest.java:44) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 58 more -18-08-2023 15:36:41 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 21][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-21: Shutdown connection -18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 21][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ -18-08-2023 15:36:41 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 15:36:41 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient -18-08-2023 15:36:41 Thread-1 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:41 Thread-1 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:41 Thread-1 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:41 main DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:36:41 main ERROR StreamingClientProvider:94 - [SF_KAFKA_CONNECTOR] Current streaming client is invalid, recreating client -18-08-2023 15:36:41 main INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 15:36:41 main DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 Thread-2 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 15:36:41 main DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 15:36:41 main INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 15:36:41 main INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 15:36:41 main INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 15:36:41 main DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 22][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-22: Shutdown connection -18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 22][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 -18-08-2023 15:36:41 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:41 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:41 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:41 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 23][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:41 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:41 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:41 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-23: Shutdown connection -18-08-2023 15:36:41 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:41 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 23][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:41 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 -18-08-2023 15:36:42 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:42 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:42 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:42 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:42 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 24][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:42 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:42 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:42 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-24: Shutdown connection -18-08-2023 15:36:42 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:42 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 24][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:42 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 -18-08-2023 15:36:44 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:44 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:44 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:44 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:44 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 25][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:44 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:44 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:44 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-25: Shutdown connection -18-08-2023 15:36:44 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:44 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 25][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:44 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 -18-08-2023 15:36:46 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:46 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:46 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:48 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:48 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:48 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:48 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:48 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 26][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:48 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:48 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:48 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-26: Shutdown connection -18-08-2023 15:36:48 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:48 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 26][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:48 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 -18-08-2023 15:36:51 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:51 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:51 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:52 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:52 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:52 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:52 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:52 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 27][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:52 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:52 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:52 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-27: Shutdown connection -18-08-2023 15:36:52 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:52 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 27][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:52 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 -18-08-2023 15:36:56 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:36:56 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:36:56 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:36:56 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:36:56 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:36:56 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:56 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:36:56 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 28][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:36:56 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:36:56 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:36:56 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-28: Shutdown connection -18-08-2023 15:36:56 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:36:56 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 28][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:36:56 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 -18-08-2023 15:37:00 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:00 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:37:00 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:00 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:00 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 29][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:00 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:00 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:00 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-29: Shutdown connection -18-08-2023 15:37:00 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:00 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 29][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:00 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 -18-08-2023 15:37:01 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:01 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:01 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:04 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:04 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:37:04 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:04 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:04 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 30][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:04 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:04 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:04 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-30: Shutdown connection -18-08-2023 15:37:04 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:04 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 30][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:04 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 -18-08-2023 15:37:06 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:06 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:06 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:08 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:08 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:37:08 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:08 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:08 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 31][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:08 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:08 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:08 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-31: Shutdown connection -18-08-2023 15:37:08 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:08 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 31][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:08 main INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 -18-08-2023 15:37:11 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:11 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:11 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:12 main INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:12 main DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.testCloseClientConcurrency(StreamingClientConcurrencyTest.java:223) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.base/java.lang.reflect.Method.invoke(Method.java:566) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) - at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) - at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) - at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runners.Suite.runChild(Suite.java:128) - at org.junit.runners.Suite.runChild(Suite.java:27) - at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) - at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) - at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) - at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) - at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) - at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) - at org.junit.runners.ParentRunner.run(ParentRunner.java:413) - at org.junit.runner.JUnitCore.run(JUnitCore.java:137) - at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) - at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) - at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) - at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) - at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) - at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 82 more -18-08-2023 15:37:12 main INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 main DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:12 main DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 32][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:12 main DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 main DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:12 main DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-32: Shutdown connection -18-08-2023 15:37:12 main DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:12 main DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 32][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 main INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ -18-08-2023 15:37:12 main DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 15:37:12 main ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient -18-08-2023 15:37:12 Thread-2 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:12 Thread-2 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:12 Thread-2 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:12 pool-11-thread-1 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-2 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-4 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-3 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-1 ERROR StreamingClientProvider:94 - [SF_KAFKA_CONNECTOR] Current streaming client is invalid, recreating client -18-08-2023 15:37:12 pool-11-thread-6 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-5 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-7 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-8 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-10 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-9 DEBUG StreamingClientProvider:70 - [SF_KAFKA_CONNECTOR] Streaming client optimization is enabled, returning the existing streaming client if valid -18-08-2023 15:37:12 pool-11-thread-1 INFO StreamingClientHandler:46 - [SF_KAFKA_CONNECTOR] Initializing Streaming Client... -18-08-2023 15:37:12 pool-11-thread-1 DEBUG SnowflakeURL:70 - [SF_INGEST] parsed Snowflake URL=sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG SecurityManager:173 - Creating new JWT with subject SFCTEST0.TEST_KAFKA and issuer SFCTEST0.TEST_KAFKA.SHA256:lqC02Uz27tb527U7mWM2+7KmM4wLtBghUeVbZBw6s38=... -18-08-2023 15:37:12 Thread-3 DEBUG HttpUtil:374 - Starting Idle Connection Monitor Thread -18-08-2023 15:37:12 pool-11-thread-1 INFO SecurityManager:190 - Successfully created new JWT -18-08-2023 15:37:12 pool-11-thread-1 INFO RequestBuilder:282 - Creating a RequestBuilder with arguments : Account : SFCTEST0, User : TEST_KAFKA, Scheme : https, Host : sfctest0.snowflakecomputing.com, Port : 443, userAgentSuffix: null -18-08-2023 15:37:12 pool-11-thread-1 INFO StreamingIngestStage:58 - [SF_INGEST] Refresh Snowflake metadata, client=KC_CLIENT_TEST_CONNECTOR_0 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestBuilder:750 - Generate Snowpipe streaming request: endpoint=/v1/streaming/client/configure/, payload={"role":"testrole_kafka"} -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 33][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-33: Shutdown connection -18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 33][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:1 and maxRetryCount:10 -18-08-2023 15:37:12 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:12 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:12 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 34][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:12 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-34: Shutdown connection -18-08-2023 15:37:12 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:12 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 34][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:12 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:2 and maxRetryCount:10 -18-08-2023 15:37:13 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:13 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:13 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:13 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:13 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 35][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:13 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:13 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:13 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-35: Shutdown connection -18-08-2023 15:37:13 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:13 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 35][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:13 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:3 and maxRetryCount:10 -18-08-2023 15:37:15 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:15 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:15 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:15 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:15 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 36][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:15 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:15 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:15 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-36: Shutdown connection -18-08-2023 15:37:15 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:15 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 36][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:15 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:4 and maxRetryCount:10 -18-08-2023 15:37:17 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:17 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:17 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:19 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:19 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:19 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:19 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:19 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 37][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:19 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:19 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:19 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-37: Shutdown connection -18-08-2023 15:37:19 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:19 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 37][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:19 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:5 and maxRetryCount:10 -18-08-2023 15:37:22 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:22 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:22 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:23 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:23 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:23 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:23 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:23 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 38][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:23 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:23 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:23 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-38: Shutdown connection -18-08-2023 15:37:23 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:23 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 38][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:23 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:6 and maxRetryCount:10 -18-08-2023 15:37:27 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:27 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:27 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:27 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:27 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:27 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:27 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:27 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 39][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:27 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:27 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:27 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-39: Shutdown connection -18-08-2023 15:37:27 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:27 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 39][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:27 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:7 and maxRetryCount:10 -18-08-2023 15:37:31 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:31 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:31 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:31 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:31 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 40][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:31 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:31 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:31 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-40: Shutdown connection -18-08-2023 15:37:31 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:31 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 40][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:31 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:8 and maxRetryCount:10 -18-08-2023 15:37:32 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:32 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:32 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:35 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:35 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:35 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:35 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:35 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 41][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:35 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:35 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:35 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-41: Shutdown connection -18-08-2023 15:37:35 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:35 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 41][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:35 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:9 and maxRetryCount:10 -18-08-2023 15:37:37 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:37 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:37 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:37:39 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:39 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:39 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:39 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:39 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 42][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:39 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:39 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:39 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-42: Shutdown connection -18-08-2023 15:37:39 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:39 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 42][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:39 pool-11-thread-1 INFO HttpUtil:298 - Retrying request which caused net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException with URI:/v1/streaming/client/configure/, retryCount:10 and maxRetryCount:10 -18-08-2023 15:37:42 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:42 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:42 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:37:43 pool-11-thread-1 INFO RetryExec:97 - I/O exception (net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException) caught when processing request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -18-08-2023 15:37:43 pool-11-thread-1 DEBUG RetryExec:104 - Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) -net.snowflake.client.jdbc.internal.apache.http.conn.HttpHostConnectException: Connect to 127.0.0.1:3128 [/127.0.0.1] failed: Connection refused (Connection refused) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:156) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(PoolingHttpClientConnectionManager.java:376) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.establishRoute(MainClientExec.java:401) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:236) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RetryExec.execute(RetryExec.java:89) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.ServiceUnavailableRetryExec.execute(ServiceUnavailableRetryExec.java:85) - at net.snowflake.client.jdbc.internal.apache.http.impl.execchain.RedirectExec.execute(RedirectExec.java:110) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) - at net.snowflake.client.jdbc.internal.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:108) - at net.snowflake.ingest.streaming.internal.StreamingIngestUtils.executeWithRetries(StreamingIngestUtils.java:130) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.makeClientConfigureCall(StreamingIngestStage.java:342) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:230) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.refreshSnowflakeMetadata(StreamingIngestStage.java:206) - at net.snowflake.ingest.streaming.internal.StreamingIngestStage.(StreamingIngestStage.java:106) - at net.snowflake.ingest.streaming.internal.FlushService.(FlushService.java:167) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:193) - at net.snowflake.ingest.streaming.internal.SnowflakeStreamingIngestClientInternal.(SnowflakeStreamingIngestClientInternal.java:221) - at net.snowflake.ingest.streaming.SnowflakeStreamingIngestClientFactory$Builder.build(SnowflakeStreamingIngestClientFactory.java:52) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler.createClient(StreamingClientHandler.java:85) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient$accessor$xUCCNIIR(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635$auxiliary$McCtLa5n.call(Unknown Source) - at org.mockito.internal.invocation.RealMethod$FromCallable$1.call(RealMethod.java:40) - at org.mockito.internal.invocation.RealMethod$FromBehavior.invoke(RealMethod.java:62) - at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:127) - at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:43) - at org.mockito.Answers.answer(Answers.java:100) - at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:106) - at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29) - at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:35) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:63) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:49) - at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor$DispatcherDefaultingToRealMethod.interceptSuperCallable(MockMethodInterceptor.java:110) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientHandler$MockitoMock$72376635.createClient(Unknown Source) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientProvider.getClient(StreamingClientProvider.java:95) - at com.snowflake.kafka.connector.internal.streaming.StreamingClientConcurrencyTest.lambda$callGetClientThread$0(StreamingClientConcurrencyTest.java:253) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) - at java.base/java.lang.Thread.run(Thread.java:829) -Caused by: java.net.ConnectException: Connection refused (Connection refused) - at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) - at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:412) - at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:255) - at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:237) - at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) - at java.base/java.net.Socket.connect(Socket.java:609) - at net.snowflake.client.jdbc.internal.apache.http.conn.socket.PlainConnectionSocketFactory.connectSocket(PlainConnectionSocketFactory.java:75) - at net.snowflake.client.jdbc.internal.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(DefaultHttpClientConnectionOperator.java:142) - ... 40 more -18-08-2023 15:37:43 pool-11-thread-1 INFO RetryExec:113 - Retrying request to {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:43 pool-11-thread-1 DEBUG RequestAddCookies:123 - CookieSpec selected: default -18-08-2023 15:37:43 pool-11-thread-1 DEBUG RequestAuthCache:77 - Auth cache not set in the context -18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:267 - Connection request: [route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:312 - Connection leased: [id: 43][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 1 of 100; total allocated: 1 of 100] -18-08-2023 15:37:43 pool-11-thread-1 DEBUG MainClientExec:234 - Opening connection {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:37:43 pool-11-thread-1 DEBUG DefaultHttpClientConnectionOperator:139 - Connecting to /127.0.0.1:3128 -18-08-2023 15:37:43 pool-11-thread-1 DEBUG DefaultManagedHttpClientConnection:96 - http-outgoing-43: Shutdown connection -18-08-2023 15:37:43 pool-11-thread-1 DEBUG MainClientExec:129 - Connection discarded -18-08-2023 15:37:43 pool-11-thread-1 DEBUG PoolingHttpClientConnectionManager:351 - Connection released: [id: 43][route: {tls}->http://127.0.0.1:3128->https://sfctest0.snowflakecomputing.com:443][total available: 0; route allocated: 0 of 100; total allocated: 0 of 100] -18-08-2023 15:37:43 pool-11-thread-1 INFO HttpUtil:291 - Max retry exceeded for requestURI:/v1/streaming/client/configure/ -18-08-2023 15:37:43 pool-11-thread-1 DEBUG HttpUtil:408 - Shutdown Idle Connection Monitor Thread -18-08-2023 15:37:43 pool-11-thread-1 ERROR StreamingClientHandler:94 - [SF_KAFKA_CONNECTOR] Exception creating streamingIngestClient -18-08-2023 15:37:43 Thread-3 DEBUG HttpUtil:391 - [IdleConnectionMonitorThread] Pool Stats: -Total Pool Stats = [leased: 0; pending: 0; available: 0; max: 100] -Pool Stats for route https://sfctest0.snowflakecomputing.com:443 = [leased: 0; pending: 0; available: 0; max: 100] - -18-08-2023 15:37:43 Thread-3 DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:37:43 Thread-3 DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 30 SECONDS -18-08-2023 15:38:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:38:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:39:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:39:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:40:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:40:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:41:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:41:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:42:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:42:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:43:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:43:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:44:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:44:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:45:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:448 - Closing expired connections -18-08-2023 15:45:39 Connection evictor DEBUG PoolingHttpClientConnectionManager:441 - Closing connections idle longer than 60000 MILLISECONDS -18-08-2023 15:46:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 -18-08-2023 15:46:36 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1978826340154525689_0 -18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1978826340154525689 -18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 -18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1978826340154525689 dropped -18-08-2023 15:46:36 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1978826340154525689_0 dropped -18-08-2023 15:46:36 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 -18-08-2023 15:46:37 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6792973406562441213_0 -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6792973406562441213 -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6792973406562441213 dropped -18-08-2023 15:46:37 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6792973406562441213_0 dropped -18-08-2023 15:46:37 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 -18-08-2023 15:46:38 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1110551375533109079_0 -18-08-2023 15:46:38 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:38 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:38 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:38 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1110551375533109079 -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1110551375533109079 dropped -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1110551375533109079_0 dropped -18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:38 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:38 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 -18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6012288537030407474_0 -18-08-2023 15:46:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6012288537030407474 -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6012288537030407474 dropped -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6012288537030407474_0 dropped -18-08-2023 15:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:39 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:39 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 -18-08-2023 15:46:39 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3110947853332713949_0 -18-08-2023 15:46:39 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:39 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3110947853332713949 -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3110947853332713949 dropped -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3110947853332713949_0 dropped -18-08-2023 15:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:40 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:40 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 -18-08-2023 15:46:40 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4233424965819834485_0 -18-08-2023 15:46:40 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:40 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:40 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:40 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4233424965819834485 -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4233424965819834485 dropped -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4233424965819834485_0 dropped -18-08-2023 15:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:41 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 -18-08-2023 15:46:41 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3313146833192302265_0 -18-08-2023 15:46:41 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3313146833192302265 -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3313146833192302265 dropped -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3313146833192302265_0 dropped -18-08-2023 15:46:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:42 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 -18-08-2023 15:46:42 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2332824327087748017_0 -18-08-2023 15:46:42 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_2332824327087748017 -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_2332824327087748017 dropped -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_2332824327087748017_0 dropped -18-08-2023 15:46:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:43 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 -18-08-2023 15:46:43 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4326952369058869337_0 -18-08-2023 15:46:43 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:43 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:43 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:43 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4326952369058869337 -18-08-2023 15:46:43 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4326952369058869337 dropped -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4326952369058869337_0 dropped -18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 -18-08-2023 15:46:44 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8264379696022517270_0 -18-08-2023 15:46:44 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:44 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:44 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:44 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8264379696022517270 -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8264379696022517270 dropped -18-08-2023 15:46:44 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8264379696022517270_0 dropped -18-08-2023 15:46:44 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:45 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 -18-08-2023 15:46:45 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8070171452629621824_0 -18-08-2023 15:46:45 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8070171452629621824 -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8070171452629621824 dropped -18-08-2023 15:46:45 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8070171452629621824_0 dropped -18-08-2023 15:46:45 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:46 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 -18-08-2023 15:46:46 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7092255300802014642_0 -18-08-2023 15:46:46 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_7092255300802014642 -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_7092255300802014642 dropped -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_7092255300802014642_0 dropped -18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:46 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:47 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:47 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:47 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:46:47 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:46:47 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:48 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:46:48 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:46:48 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:46:48 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:18 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:18 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:19 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 -18-08-2023 15:49:19 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4349441731007330241_0 -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4349441731007330241 -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4349441731007330241 dropped -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4349441731007330241_0 dropped -18-08-2023 15:49:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:20 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:20 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:20 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 -18-08-2023 15:49:20 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5888119348378217666_0 -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5888119348378217666 -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5888119348378217666 dropped -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5888119348378217666_0 dropped -18-08-2023 15:49:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:21 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:21 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 -18-08-2023 15:49:21 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6617430036033989156_0 -18-08-2023 15:49:21 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:21 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:21 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:22 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6617430036033989156 -18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 -18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6617430036033989156 dropped -18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6617430036033989156_0 dropped -18-08-2023 15:49:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:22 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:22 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 -18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4976427781984260095_0 -18-08-2023 15:49:23 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4976427781984260095 -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4976427781984260095 dropped -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4976427781984260095_0 dropped -18-08-2023 15:49:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:23 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:23 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 -18-08-2023 15:49:23 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4606067486270226287_0 -18-08-2023 15:49:23 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:23 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4606067486270226287 -18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 -18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4606067486270226287 dropped -18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4606067486270226287_0 dropped -18-08-2023 15:49:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:24 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:24 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 -18-08-2023 15:49:25 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5267653630133644600_0 -18-08-2023 15:49:25 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:25 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:25 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:25 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5267653630133644600 -18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 -18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5267653630133644600 dropped -18-08-2023 15:49:25 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5267653630133644600_0 dropped -18-08-2023 15:49:25 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 -18-08-2023 15:49:26 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4965523502332819922_0 -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4965523502332819922 -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4965523502332819922 dropped -18-08-2023 15:49:26 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4965523502332819922_0 dropped -18-08-2023 15:49:26 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 -18-08-2023 15:49:27 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1463553278395633792_0 -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1463553278395633792 -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1463553278395633792 dropped -18-08-2023 15:49:27 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1463553278395633792_0 dropped -18-08-2023 15:49:27 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 -18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4038668653217555244_0 -18-08-2023 15:49:28 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_4038668653217555244 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4038668653217555244 dropped -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4038668653217555244_0 dropped -18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:28 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 -18-08-2023 15:49:28 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6612709555054720227_0 -18-08-2023 15:49:28 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:28 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6612709555054720227 -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6612709555054720227 dropped -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6612709555054720227_0 dropped -18-08-2023 15:49:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:29 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:29 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 -18-08-2023 15:49:29 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3762733667586168679_0 -18-08-2023 15:49:29 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_3762733667586168679 -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_3762733667586168679 dropped -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_3762733667586168679_0 dropped -18-08-2023 15:49:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:30 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:30 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 -18-08-2023 15:49:30 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5874843966951528984_0 -18-08-2023 15:49:30 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5874843966951528984 -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5874843966951528984 dropped -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5874843966951528984_0 dropped -18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:31 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:31 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:31 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:32 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:32 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:32 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:32 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:32 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:32 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:33 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:33 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:33 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:33 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:49:33 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:33 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:33 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:33 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:34 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:34 main INFO InternalUtils:46 - [SF_KAFKA_CONNECTOR] Using provided role testrole_kafka for JDBC connection. -18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:49:34 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:49:34 main INFO StringConverterConfig:376 - StringConverterConfig values: - converter.encoding = UTF-8 - converter.type = value - -18-08-2023 15:49:34 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:07 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:07 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:08 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 -18-08-2023 15:51:08 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698054729379700747_0 -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5698054729379700747 -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698054729379700747 dropped -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698054729379700747_0 dropped -18-08-2023 15:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:09 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:09 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 -18-08-2023 15:51:09 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8992954341468273479_0 -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_8992954341468273479 -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_8992954341468273479 dropped -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_8992954341468273479_0 dropped -18-08-2023 15:51:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:10 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:10 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 -18-08-2023 15:51:10 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1848613164558782106_0 -18-08-2023 15:51:10 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:10 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:10 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:10 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_1848613164558782106 -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_1848613164558782106 dropped -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_1848613164558782106_0 dropped -18-08-2023 15:51:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:11 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 -18-08-2023 15:51:11 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9013078777524223289_0 -18-08-2023 15:51:11 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:11 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:11 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:11 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:51:11 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_9013078777524223289 -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_9013078777524223289 dropped -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_9013078777524223289_0 dropped -18-08-2023 15:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:12 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 -18-08-2023 15:51:12 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698617716216278091_0 -18-08-2023 15:51:12 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:12 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:12 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:12 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:51:12 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_5698617716216278091 -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_5698617716216278091 dropped -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_5698617716216278091_0 dropped -18-08-2023 15:51:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:13 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 -18-08-2023 15:51:13 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_595050023951904600_0 -18-08-2023 15:51:13 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:13 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:13 main INFO KafkaAvroDeserializerConfig:376 - KafkaAvroDeserializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - specific.avro.reader = false - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:13 main INFO AvroDataConfig:376 - AvroDataConfig values: - connect.meta.data = true - discard.type.doc.default = false - enhanced.avro.schema.support = false - generalized.sum.type.support = false - schemas.cache.config = 1000 - scrub.invalid.names = false - -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_595050023951904600 -18-08-2023 15:51:13 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_595050023951904600 dropped -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_595050023951904600_0 dropped -18-08-2023 15:51:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:14 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 -18-08-2023 15:51:14 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6259484133267789406_0 -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6259484133267789406 -18-08-2023 15:51:14 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6259484133267789406 dropped -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6259484133267789406_0 dropped -18-08-2023 15:51:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:15 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 -18-08-2023 15:51:15 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6888502239215652181_0 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create table kafka_connector_test_table_6888502239215652181 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] create stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 -18-08-2023 15:51:15 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] stage SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_6888502239215652181 dropped -18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] pipe SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_6888502239215652181_0 dropped -18-08-2023 15:51:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:16 main DEBUG SnowflakeURL:70 - [SF_KAFKA_CONNECTOR] parsed Snowflake URL: sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] Establishing a JDBC connection with url:jdbc:snowflake://sfctest0.snowflakecomputing.com:443 -18-08-2023 15:51:16 main INFO SnowflakeConnectionServiceV1:46 - [SF_KAFKA_CONNECTOR] initialized the snowflake connection -18-08-2023 15:51:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated stage name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_STAGE_kafka_connector_test_table_4542948488367988717 -18-08-2023 15:51:16 main DEBUG Utils:70 - [SF_KAFKA_CONNECTOR] generated pipe name: SNOWFLAKE_KAFKA_CONNECTOR_TEST_CONNECTOR_PIPE_kafka_connector_test_table_4542948488367988717_0 -18-08-2023 15:51:16 main INFO AvroConverterConfig:376 - AvroConverterConfig values: - auto.register.schemas = true - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - -18-08-2023 15:51:16 main INFO KafkaAvroSerializerConfig:376 - KafkaAvroSerializerConfig values: - auto.register.schemas = true - avro.reflection.allow.null = false - avro.remove.java.properties = false - avro.use.logical.type.converters = false - basic.auth.credentials.source = URL - basic.auth.user.info = [hidden] - bearer.auth.credentials.source = STATIC_TOKEN - bearer.auth.token = [hidden] - context.name.strategy = class io.confluent.kafka.serializers.context.NullContextNameStrategy - id.compatibility.strict = true - key.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - latest.compatibility.strict = true - max.schemas.per.subject = 1000 - normalize.schemas = false - proxy.host = - proxy.port = -1 - schema.reflection = false - schema.registry.basic.auth.user.info = [hidden] - schema.registry.ssl.cipher.suites = null - schema.registry.ssl.enabled.protocols = [TLSv1.2, TLSv1.3] - schema.registry.ssl.endpoint.identification.algorithm = https - schema.registry.ssl.engine.factory.class = null - schema.registry.ssl.key.password = null - schema.registry.ssl.keymanager.algorithm = SunX509 - schema.registry.ssl.keystore.certificate.chain = null - schema.registry.ssl.keystore.key = null - schema.registry.ssl.keystore.location = null - schema.registry.ssl.keystore.password = null - schema.registry.ssl.keystore.type = JKS - schema.registry.ssl.protocol = TLSv1.3 - schema.registry.ssl.provider = null - schema.registry.ssl.secure.random.implementation = null - schema.registry.ssl.trustmanager.algorithm = PKIX - schema.registry.ssl.truststore.certificates = null - schema.registry.ssl.truststore.location = null - schema.registry.ssl.truststore.password = null - schema.registry.ssl.truststore.type = JKS - schema.registry.url = [http://fake-url] - use.latest.version = false - use.schema.id = -1 - value.subject.name.strategy = class io.confluent.kafka.serializers.subject.TopicNameStrategy - diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java index b28ac882f..da4f6dc5b 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java @@ -709,7 +709,7 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { .forEach( converter -> { config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.toString()); + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); Utils.validateConfig(config); }); @@ -717,7 +717,7 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { .forEach( converter -> { config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.toString()); + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.name); Utils.validateConfig(config); }); } @@ -735,7 +735,7 @@ public void testInvalidKeyConvertersForStreamingSnowpipe() { converter -> { try { config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.toString()); + SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); @@ -766,7 +766,7 @@ public void testInvalidValueConvertersForStreamingSnowpipe() { "org.apache.kafka.connect.storage.StringConverter"); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - converter.toString()); + converter.name); Utils.validateConfig(config); } catch (SnowflakeKafkaConnectorException exception) { diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 9883b8b05..8bc59fbb6 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -235,257 +235,4 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { service.closeAll(); } - - // - // @Test - // public void testTombstoneRecords_DEFAULT_behavior_ingestion_SFJsonConverter() throws Exception - // { - // conn.createTable(table); - // conn.createStage(stage); - // - // Map connectorConfig = new HashMap<>(); - // connectorConfig.put( - // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - // - // SnowflakeSinkService service = - // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, - // connectorConfig) - // .setRecordNumber(1) - // .addTask(table, new TopicPartition(topic, partition0)) - // .build(); - // - // SnowflakeConverter converter = new SnowflakeJsonConverter(); - // SchemaAndValue input = converter.toConnectData(topic, null); - // long offset = 0; - // - // SinkRecord record1 = - // new SinkRecord( - // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), - // offset); - // service.insert(Collections.singletonList(record1)); - // TestUtils.assertWithRetry( - // () -> - // conn.listStage( - // stage, - // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) - // .size() - // == 1, - // 5, - // 4); - // service.callAllGetOffset(); - // List files = - // conn.listStage( - // stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); - // String fileName = files.get(0); - // - // assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); - // assert FileNameUtils.fileNameToPartition(fileName) == partition0; - // assert FileNameUtils.fileNameToStartOffset(fileName) == offset; - // assert FileNameUtils.fileNameToEndOffset(fileName) == offset; - // - // // wait for ingest - // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); - // - // ResultSet resultSet = TestUtils.showTable(table); - // LinkedList contentResult = new LinkedList<>(); - // while (resultSet.next()) { - // contentResult.add(resultSet.getString("RECORD_CONTENT")); - // } - // resultSet.close(); - // - // assert contentResult.size() == 1; - // - // ObjectNode emptyNode = MAPPER.createObjectNode(); - // assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); - // - // // change cleaner - // TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); - // - // assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; - // - // service.closeAll(); - // } - // - // @Test - // public void testTombstoneRecords_IGNORE_behavior_ingestion_SFJsonConverter() throws Exception - // { - // conn.createTable(table); - // conn.createStage(stage); - // - // Map connectorConfig = new HashMap<>(); - // connectorConfig.put( - // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - // String.valueOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE)); - // - // SnowflakeSinkService service = - // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, - // connectorConfig) - // .setRecordNumber(1) - // .addTask(table, new TopicPartition(topic, partition0)) - // .build(); - // - // SnowflakeConverter converter = new SnowflakeJsonConverter(); - // SchemaAndValue input = converter.toConnectData(topic, null); - // long offset = 0; - // - // SinkRecord record1 = - // new SinkRecord( - // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), - // offset); - // service.insert(Collections.singletonList(record1)); - // Assert.assertTrue( - // ((SnowflakeSinkServiceV1) service) - // .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); - // TestUtils.assertWithRetry( - // () -> - // conn.listStage( - // stage, - // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) - // .size() - // == 0, - // 5, - // 4); - // - // // wait for ingest - // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); - // - // ResultSet resultSet = TestUtils.showTable(table); - // Assert.assertTrue(resultSet.getFetchSize() == 0); - // resultSet.close(); - // - // service.closeAll(); - // } - // - // @Test - // public void testTombstoneRecords_DEFAULT_behavior_ingestion_CommunityJsonConverter() - // throws Exception { - // conn.createTable(table); - // conn.createStage(stage); - // - // Map connectorConfig = new HashMap<>(); - // connectorConfig.put( - // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - // - // SnowflakeSinkService service = - // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, - // connectorConfig) - // .setRecordNumber(1) - // .addTask(table, new TopicPartition(topic, partition0)) - // .build(); - // - // JsonConverter converter = new JsonConverter(); - // HashMap converterConfig = new HashMap(); - // converterConfig.put("schemas.enable", "false"); - // converter.configure(converterConfig, false); - // SchemaAndValue input = converter.toConnectData(topic, null); - // long offset = 0; - // - // SinkRecord record1 = - // new SinkRecord( - // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), - // offset); - // service.insert(Collections.singletonList(record1)); - // TestUtils.assertWithRetry( - // () -> - // conn.listStage( - // stage, - // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) - // .size() - // == 1, - // 5, - // 4); - // service.callAllGetOffset(); - // List files = - // conn.listStage( - // stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)); - // String fileName = files.get(0); - // - // assert FileNameUtils.fileNameToTimeIngested(fileName) < System.currentTimeMillis(); - // assert FileNameUtils.fileNameToPartition(fileName) == partition0; - // assert FileNameUtils.fileNameToStartOffset(fileName) == offset; - // assert FileNameUtils.fileNameToEndOffset(fileName) == offset; - // - // // wait for ingest - // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); - // - // ResultSet resultSet = TestUtils.showTable(table); - // LinkedList contentResult = new LinkedList<>(); - // while (resultSet.next()) { - // contentResult.add(resultSet.getString("RECORD_CONTENT")); - // } - // resultSet.close(); - // - // assert contentResult.size() == 1; - // - // ObjectNode emptyNode = new ObjectMapper().createObjectNode(); - // assert contentResult.get(0).equalsIgnoreCase(emptyNode.toString()); - // - // // change cleaner - // TestUtils.assertWithRetry(() -> getStageSize(stage, table, partition0) == 0, 30, 20); - // - // assert service.getOffset(new TopicPartition(topic, partition0)) == offset + 1; - // - // service.closeAll(); - // } - // - // @Test - // public void testTombstoneRecords_IGNORE_behavior_ingestion_CommunityJsonConverter() - // throws Exception { - // conn.createTable(table); - // conn.createStage(stage); - // - // Map connectorConfig = new HashMap<>(); - // connectorConfig.put( - // SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - // SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - // - // SnowflakeSinkService service = - // SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, - // connectorConfig) - // .setRecordNumber(1) - // .addTask(table, new TopicPartition(topic, partition0)) - // .build(); - // - // JsonConverter converter = new JsonConverter(); - // HashMap converterConfig = new HashMap(); - // converterConfig.put("schemas.enable", "false"); - // converter.configure(converterConfig, false); - // SchemaAndValue input = converter.toConnectData(topic, null); - // long offset = 0; - // - // SinkRecord record1 = - // new SinkRecord( - // topic, partition0, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), - // offset); - // service.insert(Collections.singletonList(record1)); - // Assert.assertTrue( - // ((SnowflakeSinkServiceV1) service) - // .isPartitionBufferEmpty(SnowflakeSinkServiceV1.getNameIndex(topic, partition0))); - // TestUtils.assertWithRetry( - // () -> - // conn.listStage( - // stage, - // FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) - // .size() - // == 0, - // 5, - // 4); - // - // // wait for ingest - // TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 0, 30, 20); - // - // ResultSet resultSet = TestUtils.showTable(table); - // Assert.assertTrue(resultSet.getFetchSize() == 0); - // resultSet.close(); - // - // service.closeAll(); - // } - - int getStageSize(String stage, String table, int partition0) { - return conn.listStage( - stage, FileNameUtils.filePrefix(TestUtils.TEST_CONNECTOR_NAME, table, partition0)) - .size(); - } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index cb561c5a9..d8db659d6 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -1356,49 +1356,6 @@ public void testStreamingIngestion_invalid_file_version() throws Exception { Assert.assertEquals(NumberFormatException.class, ex.getCause().getClass()); } } - - @Test - public void testTombstoneIngestion() throws Exception { - Map config = TestUtils.getConfForStreaming(); - SnowflakeSinkConnectorConfig.setDefaultValues(config); - config.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - - conn.createTable(table); - - // opens a channel for partition 0, table and topic - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config) - .setRecordNumber(1) - .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) - .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) - .addTask(table, new TopicPartition(topic, partition)) // Internally calls startTask - .build(); - - SnowflakeConverter converter = new SnowflakeJsonConverter(); - SchemaAndValue input = - converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); - long offset = 0; - - SinkRecord record1 = - new SinkRecord( - topic, - partition, - Schema.STRING_SCHEMA, - "test_key" + offset, - input.schema(), - null, - offset); - - service.insert(record1); - - TestUtils.assertWithRetry( - () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); - - service.closeAll(); - } - private void createNonNullableColumn(String tableName, String colName) { String createTableQuery = "alter table identifier(?) add " + colName + " int not null"; From 1dde035c735d638e40048a2a8a6d110d773851a3 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 15:48:47 -0700 Subject: [PATCH 11/42] autoformatting --- .../kafka/connector/ConnectorConfigTest.java | 12 ++++-------- .../internal/streaming/SnowflakeSinkServiceV2IT.java | 1 + 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java index da4f6dc5b..395b80b7c 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java @@ -708,16 +708,14 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { Arrays.stream(CommunityConverterSubset.values()) .forEach( converter -> { - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); + config.put(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); Utils.validateConfig(config); }); Arrays.stream(CommunityConverterSubset.values()) .forEach( converter -> { - config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.name); + config.put(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.name); Utils.validateConfig(config); }); } @@ -734,8 +732,7 @@ public void testInvalidKeyConvertersForStreamingSnowpipe() { .forEach( converter -> { try { - config.put( - SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); + config.put(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.name); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); @@ -765,8 +762,7 @@ public void testInvalidValueConvertersForStreamingSnowpipe() { SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); config.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, - converter.name); + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.name); Utils.validateConfig(config); } catch (SnowflakeKafkaConnectorException exception) { diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index d8db659d6..2aaf85159 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -1356,6 +1356,7 @@ public void testStreamingIngestion_invalid_file_version() throws Exception { Assert.assertEquals(NumberFormatException.class, ex.getCause().getClass()); } } + private void createNonNullableColumn(String tableName, String colName) { String createTableQuery = "alter table identifier(?) add " + colName + " int not null"; From 62e475715670b8cbe0b75ba13c7b1ef3eeb8ac01 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 16:34:43 -0700 Subject: [PATCH 12/42] eyeball cc --- .../com/snowflake/kafka/connector/Utils.java | 13 +++++ .../internal/SnowflakeSinkServiceV1.java | 14 +---- .../streaming/SnowflakeSinkServiceV2.java | 14 +---- .../connector/records/RecordService.java | 20 +++++-- .../snowflake/kafka/connector/UtilsTest.java | 18 ++++++ .../internal/TombstoneRecordIngestionIT.java | 56 +++++++++---------- .../connector/records/RecordContentTest.java | 23 ++++++++ 7 files changed, 101 insertions(+), 57 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/Utils.java b/src/main/java/com/snowflake/kafka/connector/Utils.java index 21dbb2fea..ac5d29d8d 100644 --- a/src/main/java/com/snowflake/kafka/connector/Utils.java +++ b/src/main/java/com/snowflake/kafka/connector/Utils.java @@ -724,6 +724,19 @@ public static String getExceptionMessage(String customMessage, Exception ex) { return formatString(GET_EXCEPTION_FORMAT, customMessage, message, cause); } + public static SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesEnum(Map config) { + return Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) + .filter( + behavior -> + behavior + .toString() + .equalsIgnoreCase( + config.get( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + .findAny() + .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + } + private static void handleInvalidParameters(ImmutableMap invalidConfigParams) { // log all invalid params and throw exception if (!invalidConfigParams.isEmpty()) { diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index 2ffce054b..eccb7ee09 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -93,17 +93,7 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { // Setting the default value in constructor // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = - Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) - .filter( - behavior -> - behavior - .toString() - .equalsIgnoreCase( - connectorConfig.get( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) - .findAny() - .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + this.behaviorOnNullValues = Utils.getBehaviorOnNullValuesEnum(connectorConfig); } /** @@ -140,7 +130,7 @@ public void insert(final Collection records) { // note that records can be empty for (SinkRecord record : records) { // check if need to handle null value records - if (recordService.shouldSkipNullValue(record, behaviorOnNullValues)) { + if (recordService.shouldSkipNullValue(record)) { continue; } // Might happen a count of record based flushing diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index f91a117f4..5286e1cab 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -126,17 +126,7 @@ public SnowflakeSinkServiceV2( // Setting the default value in constructor // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = - Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) - .filter( - behavior -> - behavior - .toString() - .equalsIgnoreCase( - connectorConfig.get( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) - .findAny() - .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + this.behaviorOnNullValues = Utils.getBehaviorOnNullValuesEnum(connectorConfig); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() @@ -276,7 +266,7 @@ public void insert(Collection records) { // note that records can be empty but, we will still need to check for time based flush for (SinkRecord record : records) { // check if need to handle null value records - if (recordService.shouldSkipNullValue(record, behaviorOnNullValues)) { + if (recordService.shouldSkipNullValue(record)) { continue; } // While inserting into buffer, we will check for count threshold and buffered bytes diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index fd5eba993..079954907 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -126,7 +126,7 @@ public RecordService(boolean enableSchematization, boolean ingestTombstoneRecord public RecordService() {} public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { - metadataConfig = metadataConfigIn; + this.metadataConfig = metadataConfigIn; } /** @@ -506,15 +506,13 @@ else if (value instanceof ByteBuffer) { *

If the value is an empty JSON node, we could assume the value passed was null. * * @param record record sent from Kafka to KC - * @param behaviorOnNullValues behavior passed inside KC * @return true if we would skip adding it to buffer * @see com.snowflake.kafka.connector.records.SnowflakeJsonConverter#toConnectData when bytes == * null case */ public boolean shouldSkipNullValue( - SinkRecord record, - final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues) { - if (behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { + SinkRecord record) { + if (this.ingestTombstoneRecords) { return false; } else { boolean isRecordValueNull = false; @@ -560,4 +558,16 @@ public boolean shouldSkipNullValue( } return false; } + + // TESTING ONLY + @VisibleForTesting + public boolean getEnableSchematization() { + return this.enableSchematization; + } + + // TESTING ONLY + @VisibleForTesting + public boolean getIngestTombstoneRecords() { + return this.ingestTombstoneRecords; + } } diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index 3c831966b..5b15c7d75 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -279,4 +279,22 @@ public void testGetExceptionMessage() throws Exception { .equals( Utils.formatString(Utils.GET_EXCEPTION_FORMAT, customMessage, exceptionMessage, "[]")); } + + @Test + public void testValidGetBehaviorOnNullValuesEnum() { + Map config = new HashMap<>(); + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE; + config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior.toString()); + + assert Utils.getBehaviorOnNullValuesEnum(config).equals(behavior); + } + + + @Test + public void testInvalidGetBehaviorOnNullValuesEnum() { + Map config = new HashMap<>(); + config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "not expected behavior"); + + assert Utils.getBehaviorOnNullValuesEnum(config).equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 8bc59fbb6..91f822a8c 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -34,34 +34,34 @@ public static Collection input() { IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE, - ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE_STREAMING, - ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter - }, - { - IngestionMethodConfig.SNOWPIPE_STREAMING, - ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter - }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE, +// ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE_STREAMING, +// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter +// }, +// { +// IngestionMethodConfig.SNOWPIPE_STREAMING, +// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter +// }, { IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 505db2941..e8551cb8b 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -1,5 +1,6 @@ package com.snowflake.kafka.connector.records; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; import com.snowflake.kafka.connector.internal.TestUtils; @@ -9,6 +10,7 @@ import java.util.Arrays; import java.util.Base64; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonProcessingException; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.type.TypeReference; @@ -20,6 +22,8 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; public class RecordContentTest { private ObjectMapper mapper = new ObjectMapper(); @@ -271,4 +275,23 @@ public void testColumnNameFormatting() throws JsonProcessingException { assert got.containsKey("\"NaMe\""); assert got.containsKey("AnSwEr"); } + + @Test + public void testRecordServiceConfigCreation() { + this.testRecordServiceConfigCreationRunner(true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + this.testRecordServiceConfigCreationRunner(false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + this.testRecordServiceConfigCreationRunner(true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + this.testRecordServiceConfigCreationRunner(false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + } + + private void testRecordServiceConfigCreationRunner(boolean enableSchematization, String behavior) { + Map config = new HashMap<>(); + + config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, String.valueOf(enableSchematization)); + config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior); + + RecordService recordService = new RecordService(config); + + assert recordService.getEnableSchematization() == enableSchematization && recordService.getIngestTombstoneRecords() == (behavior.equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + } } From 94ac2ca2680442b0d575e62a1035c9796ef30d7f Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 16:35:06 -0700 Subject: [PATCH 13/42] uncomment test cases --- .../internal/TombstoneRecordIngestionIT.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 91f822a8c..8bc59fbb6 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -34,34 +34,34 @@ public static Collection input() { IngestionMethodConfig.SNOWPIPE, ConnectorConfigTest.CustomSfConverter.JSON_CONVERTER.converter }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE, -// ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE_STREAMING, -// ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter -// }, -// { -// IngestionMethodConfig.SNOWPIPE_STREAMING, -// ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter -// }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CustomSfConverter.AVRO_CONVERTER_WITHOUT_SCHEMA_REGISTRY.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE, + ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.JSON_CONVERTER.converter + }, + { + IngestionMethodConfig.SNOWPIPE_STREAMING, + ConnectorConfigTest.CommunityConverterSubset.AVRO_CONVERTER.converter + }, { IngestionMethodConfig.SNOWPIPE_STREAMING, ConnectorConfigTest.CommunityConverterSubset.STRING_CONVERTER.converter From e73a0638a82f5720086a2b80cefe67156e77cc16 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 16:35:18 -0700 Subject: [PATCH 14/42] autoformatting --- .../com/snowflake/kafka/connector/Utils.java | 6 ++--- .../streaming/SnowflakeSinkServiceV2.java | 1 - .../connector/records/RecordService.java | 3 +-- .../snowflake/kafka/connector/UtilsTest.java | 10 ++++--- .../connector/records/RecordContentTest.java | 26 ++++++++++++------- 5 files changed, 27 insertions(+), 19 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/Utils.java b/src/main/java/com/snowflake/kafka/connector/Utils.java index ac5d29d8d..6abf038eb 100644 --- a/src/main/java/com/snowflake/kafka/connector/Utils.java +++ b/src/main/java/com/snowflake/kafka/connector/Utils.java @@ -724,15 +724,15 @@ public static String getExceptionMessage(String customMessage, Exception ex) { return formatString(GET_EXCEPTION_FORMAT, customMessage, message, cause); } - public static SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesEnum(Map config) { + public static SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesEnum( + Map config) { return Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) .filter( behavior -> behavior .toString() .equalsIgnoreCase( - config.get( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) + config.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) .findAny() .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); } diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 5286e1cab..44274284d 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -18,7 +18,6 @@ import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import com.snowflake.kafka.connector.records.RecordService; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.Map; diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 079954907..a8fca6a56 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -510,8 +510,7 @@ else if (value instanceof ByteBuffer) { * @see com.snowflake.kafka.connector.records.SnowflakeJsonConverter#toConnectData when bytes == * null case */ - public boolean shouldSkipNullValue( - SinkRecord record) { + public boolean shouldSkipNullValue(SinkRecord record) { if (this.ingestTombstoneRecords) { return false; } else { diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index 5b15c7d75..019fb2a56 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -283,18 +283,20 @@ public void testGetExceptionMessage() throws Exception { @Test public void testValidGetBehaviorOnNullValuesEnum() { Map config = new HashMap<>(); - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE; + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE; config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior.toString()); assert Utils.getBehaviorOnNullValuesEnum(config).equals(behavior); } - @Test public void testInvalidGetBehaviorOnNullValuesEnum() { Map config = new HashMap<>(); - config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "not expected behavior"); + config.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "not expected behavior"); - assert Utils.getBehaviorOnNullValuesEnum(config).equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); + assert Utils.getBehaviorOnNullValuesEnum(config) + .equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); } } diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index e8551cb8b..e0d9b2898 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -22,8 +22,6 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; public class RecordContentTest { private ObjectMapper mapper = new ObjectMapper(); @@ -278,20 +276,30 @@ public void testColumnNameFormatting() throws JsonProcessingException { @Test public void testRecordServiceConfigCreation() { - this.testRecordServiceConfigCreationRunner(true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - this.testRecordServiceConfigCreationRunner(false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - this.testRecordServiceConfigCreationRunner(true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - this.testRecordServiceConfigCreationRunner(false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + this.testRecordServiceConfigCreationRunner( + true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + this.testRecordServiceConfigCreationRunner( + false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + this.testRecordServiceConfigCreationRunner( + true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + this.testRecordServiceConfigCreationRunner( + false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); } - private void testRecordServiceConfigCreationRunner(boolean enableSchematization, String behavior) { + private void testRecordServiceConfigCreationRunner( + boolean enableSchematization, String behavior) { Map config = new HashMap<>(); - config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, String.valueOf(enableSchematization)); + config.put( + SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, + String.valueOf(enableSchematization)); config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior); RecordService recordService = new RecordService(config); - assert recordService.getEnableSchematization() == enableSchematization && recordService.getIngestTombstoneRecords() == (behavior.equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); + assert recordService.getEnableSchematization() == enableSchematization + && recordService.getIngestTombstoneRecords() + == (behavior.equals( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); } } From 754824ecc9ee4f383db8ea1bda77652a999bd2dd Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 17:01:15 -0700 Subject: [PATCH 15/42] autoformatting --- src/test/java/com/snowflake/kafka/connector/UtilsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index dda244d7e..a06ab8aa8 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -294,7 +294,7 @@ public void testGetSnowflakeOAuthAccessToken() { SnowflakeErrors.ERROR_1004, () -> Utils.getSnowflakeOAuthAccessToken(url, "INVALID", "INVALID", "INVALID")); } - + @Test public void testValidGetBehaviorOnNullValuesEnum() { Map config = new HashMap<>(); From 6c206563d831ace75afbd6337bf4e9390b91f8ab Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 18:44:09 -0700 Subject: [PATCH 16/42] fix sinktest --- .../com/snowflake/kafka/connector/Utils.java | 13 ------------ .../connector/internal/SnowflakeErrors.java | 14 ++++++++----- .../internal/SnowflakeSinkServiceV1.java | 7 ------- .../streaming/SnowflakeSinkServiceV2.java | 13 ------------ .../connector/records/RecordService.java | 7 ++++++- .../snowflake/kafka/connector/UtilsTest.java | 20 ------------------- 6 files changed, 15 insertions(+), 59 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/Utils.java b/src/main/java/com/snowflake/kafka/connector/Utils.java index 6abf038eb..21dbb2fea 100644 --- a/src/main/java/com/snowflake/kafka/connector/Utils.java +++ b/src/main/java/com/snowflake/kafka/connector/Utils.java @@ -724,19 +724,6 @@ public static String getExceptionMessage(String customMessage, Exception ex) { return formatString(GET_EXCEPTION_FORMAT, customMessage, message, cause); } - public static SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesEnum( - Map config) { - return Arrays.stream(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()) - .filter( - behavior -> - behavior - .toString() - .equalsIgnoreCase( - config.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG))) - .findAny() - .orElse(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); - } - private static void handleInvalidParameters(ImmutableMap invalidConfigParams) { // log all invalid params and throw exception if (!invalidConfigParams.isEmpty()) { diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index ec259dd0c..5728013bd 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -262,10 +262,7 @@ public enum SnowflakeErrors { ERROR_5016( "5016", "Invalid SinkRecord received", - "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" - + " is enabled (see " - + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG - + " for more information."), + "SinkRecord.key and SinkRecord.keySchema cannot be null"), ERROR_5017( "5017", "Invalid api call to cached put", "Cached put only support AWS, Azure and GCS."), ERROR_5018("5018", "Failed to execute cached put", "Error in cached put command"), @@ -275,7 +272,14 @@ public enum SnowflakeErrors { "5021", "Failed to get data schema", "Failed to get data schema. Unrecognizable data type in JSON object"), - ERROR_5022("5022", "Invalid column name", "Failed to find column in the schema"); + ERROR_5022("5022", "Invalid column name", "Failed to find column in the schema"), + ERROR_5023( + "5016", + "Invalid SinkRecord received", + "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" + + " is enabled (see " + + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + + " for more information."),; // properties diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index eccb7ee09..74f36089d 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -69,9 +69,6 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { private final SnowflakeTelemetryService telemetryService; private Map topic2TableMap; - // Behavior to be set at the start of connector start. (For tombstone records) - private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues; - // default is true unless the configuration provided is false; // If this is true, we will enable Mbean for required classes and emit JMX metrics for monitoring private boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; @@ -90,10 +87,6 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { this.telemetryService = conn.getTelemetryClient(); this.recordService = new RecordService(connectorConfig); this.topic2TableMap = new HashMap<>(); - - // Setting the default value in constructor - // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = Utils.getBehaviorOnNullValuesEnum(connectorConfig); } /** diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 44274284d..de47b395a 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -63,9 +63,6 @@ public class SnowflakeSinkServiceV2 implements SnowflakeSinkService { private final SnowflakeTelemetryService telemetryService; private Map topicToTableMap; - // Behavior to be set at the start of connector start. (For tombstone records) - private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues; - // default is true unless the configuration provided is false; // If this is true, we will enable Mbean for required classes and emit JMX metrics for monitoring private boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; @@ -111,11 +108,6 @@ public SnowflakeSinkServiceV2( this.telemetryService = conn.getTelemetryClient(); this.recordService = new RecordService(connectorConfig); this.topicToTableMap = new HashMap<>(); - - // Setting the default value in constructor - // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; - this.connectorConfig = connectorConfig; this.enableSchematization = @@ -123,10 +115,6 @@ public SnowflakeSinkServiceV2( connectorConfig.getOrDefault( SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); - // Setting the default value in constructor - // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) - this.behaviorOnNullValues = Utils.getBehaviorOnNullValuesEnum(connectorConfig); - this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() .getClient(this.connectorConfig); @@ -160,7 +148,6 @@ public SnowflakeSinkServiceV2( this.recordService = recordService; this.telemetryService = telemetryService; this.topicToTableMap = topicToTableMap; - this.behaviorOnNullValues = behaviorOnNullValues; this.enableCustomJMXMonitoring = enableCustomJMXMonitoring; this.kafkaRecordErrorReporter = kafkaRecordErrorReporter; this.sinkTaskContext = sinkTaskContext; diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index a8fca6a56..08fa3a4a0 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -138,11 +138,16 @@ public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { private SnowflakeTableRow processRecord(SinkRecord record) { SnowflakeRecordContent valueContent; + // cannot ingest null key + if (record.key() == null || record.keySchema() == null) { + throw SnowflakeErrors.ERROR_5016.getException(); + } + if (record.value() == null || record.valueSchema() == null) { if (this.ingestTombstoneRecords) { valueContent = new SnowflakeRecordContent(); } else { - throw SnowflakeErrors.ERROR_5016.getException(); + throw SnowflakeErrors.ERROR_5023.getException(); } } else { if (!record.valueSchema().name().equals(SnowflakeJsonSchema.NAME)) { diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index 019fb2a56..3c831966b 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -279,24 +279,4 @@ public void testGetExceptionMessage() throws Exception { .equals( Utils.formatString(Utils.GET_EXCEPTION_FORMAT, customMessage, exceptionMessage, "[]")); } - - @Test - public void testValidGetBehaviorOnNullValuesEnum() { - Map config = new HashMap<>(); - SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE; - config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior.toString()); - - assert Utils.getBehaviorOnNullValuesEnum(config).equals(behavior); - } - - @Test - public void testInvalidGetBehaviorOnNullValuesEnum() { - Map config = new HashMap<>(); - config.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "not expected behavior"); - - assert Utils.getBehaviorOnNullValuesEnum(config) - .equals(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT); - } } From b85f7339dbf992c58e94ebabc44f2d994302138c Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:13:51 -0700 Subject: [PATCH 17/42] add null key failure test --- .../connector/internal/SnowflakeErrors.java | 3 +- .../connector/records/RecordContentTest.java | 53 +++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index 5728013bd..39e7f0e52 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -279,7 +279,8 @@ public enum SnowflakeErrors { "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" + " is enabled (see " + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG - + " for more information."),; + + " for more information."), + ; // properties diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index e0d9b2898..0f5abb7b3 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -302,4 +302,57 @@ private void testRecordServiceConfigCreationRunner( == (behavior.equals( SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); } + + @Test + public void testGetProcessedRecordNullValue() throws JsonProcessingException { + RecordService service = new RecordService(false, true); + SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); + + SchemaAndValue sv = jsonConverter.toConnectData(topic, null); + String keyStr = "string"; + + SinkRecord allNullRecord = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, keyStr, null, null, partition); + SinkRecord nullValueRecord = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition); + SinkRecord nullValueSchemaRecord = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition); + + assert service.getProcessedRecordForStreamingIngest(allNullRecord).values().stream() + .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) + .count() + == 1; + assert service.getProcessedRecordForStreamingIngest(nullValueRecord).values().stream() + .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) + .count() + == 1; + assert service.getProcessedRecordForStreamingIngest(nullValueSchemaRecord).values().stream() + .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) + .count() + == 1; + } + + @Test(expected = SnowflakeKafkaConnectorException.class) + public void testGetProcessedRecordNullKeyFails() throws JsonProcessingException { + RecordService service = new RecordService(false, true); + SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); + + SchemaAndValue sv = jsonConverter.toConnectData(topic, null); + + SinkRecord allNullRecord = + new SinkRecord(topic, partition, null, null, sv.schema(), sv.value(), partition); + SinkRecord nullKeyRecord = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition); + SinkRecord nullKeySchemaRecord = + new SinkRecord(topic, partition, null, "string", sv.schema(), sv.value(), partition); + + // null key schema or key will fail + service.getProcessedRecordForStreamingIngest(allNullRecord); + service.getProcessedRecordForStreamingIngest(nullKeyRecord); + service.getProcessedRecordForStreamingIngest(nullKeySchemaRecord); + } } From 4ab7ffa08a6bf2d5b357a91fc208dfb0313d0539 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:13:55 -0700 Subject: [PATCH 18/42] autoformatting --- .../kafka/connector/records/RecordContentTest.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 0f5abb7b3..b337a1192 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -312,26 +312,24 @@ public void testGetProcessedRecordNullValue() throws JsonProcessingException { String keyStr = "string"; SinkRecord allNullRecord = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, null, null, partition); + new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, null, partition); SinkRecord nullValueRecord = new SinkRecord( topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition); SinkRecord nullValueSchemaRecord = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition); + new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition); assert service.getProcessedRecordForStreamingIngest(allNullRecord).values().stream() .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) .count() == 1; assert service.getProcessedRecordForStreamingIngest(nullValueRecord).values().stream() - .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) - .count() + .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) + .count() == 1; assert service.getProcessedRecordForStreamingIngest(nullValueSchemaRecord).values().stream() - .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) - .count() + .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) + .count() == 1; } From 37dd5a77c6a33ea47ca3320d6c3fd4ff817325dd Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:15:42 -0700 Subject: [PATCH 19/42] autoformatting --- src/test/java/com/snowflake/kafka/connector/UtilsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index 68544ccbf..3f0828a3a 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -280,7 +280,7 @@ public void testGetExceptionMessage() throws Exception { .equals( Utils.formatString(Utils.GET_EXCEPTION_FORMAT, customMessage, exceptionMessage, "[]")); } - + @Test public void testGetSnowflakeOAuthAccessToken() { Map config = TestUtils.getConfForStreamingWithOAuth(); From 65aab68f50d2f65686b4bcc7d5a4cee9f2b53485 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:34:41 -0700 Subject: [PATCH 20/42] fix dlq test --- .../kafka/connector/SnowflakeSinkTaskStreamingTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java index 3e8678645..131dfdc4e 100644 --- a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java +++ b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java @@ -126,8 +126,8 @@ public void testSinkTaskInvalidRecord_InMemoryDLQ() throws Exception { new SinkRecord( topicName, partition, - null, - null, + Schema.STRING_SCHEMA, + "String", brokenInputValue.schema(), brokenInputValue.value(), 0); From cfd6f54144d19e73de66bbf5862f92b541858c2f Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:50:19 -0700 Subject: [PATCH 21/42] add exception case --- .../connector/records/ProcessRecordTest.java | 51 +++++++++++-------- 1 file changed, 30 insertions(+), 21 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java index 7482f7af8..90df1d1ba 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java @@ -2,6 +2,8 @@ import static org.junit.Assert.assertEquals; +import com.snowflake.kafka.connector.internal.SnowflakeErrors; +import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; import com.snowflake.kafka.connector.mock.MockSchemaRegistryClient; import java.io.IOException; import java.net.URL; @@ -47,10 +49,13 @@ public void test() throws IOException { testCase.value.schema(), testCase.value.value(), partition); - - String got = service.getProcessedRecordForSnowpipe(record); - - assertEquals(testCase.expected, mapper.readTree(got)); + + try { + String got = service.getProcessedRecordForSnowpipe(record); + assertEquals(testCase.expectedJson, mapper.readTree(got)); + } catch (Exception ex) { + assert testCase.expectedException.getMessage().equals(ex.getMessage()); + } } @Parameterized.Parameters(name = "{index}: {0}") @@ -62,7 +67,7 @@ public static Iterable data() throws IOException { getAvro(), mapper.readTree( "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":\"string" - + " value\"}}")), + + " value\"}}"), null), new Case( "string key, avro without registry value", getString(), @@ -70,68 +75,68 @@ public static Iterable data() throws IOException { mapper.readTree( "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" + " value\"}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}")), + + " value\"}}"), null), new Case( "string key, json value", getString(), getJson(), mapper.readTree( "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}")), + + " value\"}}"), null), new Case( "avro key, avro value", getAvro(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}")), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}"), null), new Case( "avro key, avro without registry value", getAvro(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}")), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}"), null), new Case( "avro key, json value", getAvro(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}")), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}"), null), new Case( "avro without registry key, avro value", getAvroWithoutRegistryKey(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}")), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}"), null), new Case( "avro without registry key, avro without registry value", getAvroWithoutRegistryKey(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}")), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}"), null), new Case( "avro without registry key, json value", getAvroWithoutRegistryKey(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}")), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}"), null), new Case( "json key, avro value", getJson(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}")), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}"), null), new Case( "json key, avro without registry value", getJson(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}")), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}"), null), new Case( "json key, json value", getJson(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}")), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}"), null), new Case( "multi line avro key, multi line avro value", getAvroMultiLine(), @@ -140,13 +145,15 @@ public static Iterable data() throws IOException { "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"topic\":\"test\",\"offset\":0,\"partition\":0,\"key\":[{\"username\":\"miguno\",\"tweet\":\"Rock:" + " Nerf paper, scissors is" + " fine.\",\"timestamp\":1366150681},{\"username\":\"BlizzardCS\",\"tweet\":\"Works" - + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}")), + + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}"), null), new Case( "null key, json value", getNull(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"))); + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"), + SnowflakeErrors.ERROR_5016.getException()) + ); } public static SchemaAndValue getString() { @@ -211,13 +218,15 @@ private static class Case { String name; SchemaAndValue key; SchemaAndValue value; - JsonNode expected; + JsonNode expectedJson; + Exception expectedException; - public Case(String name, SchemaAndValue key, SchemaAndValue value, JsonNode expected) { + public Case(String name, SchemaAndValue key, SchemaAndValue value, JsonNode expectedJson, Exception expectedException) { this.name = name; this.key = key; this.value = value; - this.expected = expected; + this.expectedJson = expectedJson; + this.expectedException = expectedException; } @Override From 32d7a6008dca3c8e9a774989d23b9982196237cc Mon Sep 17 00:00:00 2001 From: revi cheng Date: Mon, 21 Aug 2023 19:50:23 -0700 Subject: [PATCH 22/42] autoformatting --- .../connector/records/ProcessRecordTest.java | 52 ++++++++++++------- 1 file changed, 34 insertions(+), 18 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java index 90df1d1ba..e04f65e29 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java @@ -3,7 +3,6 @@ import static org.junit.Assert.assertEquals; import com.snowflake.kafka.connector.internal.SnowflakeErrors; -import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; import com.snowflake.kafka.connector.mock.MockSchemaRegistryClient; import java.io.IOException; import java.net.URL; @@ -49,7 +48,7 @@ public void test() throws IOException { testCase.value.schema(), testCase.value.value(), partition); - + try { String got = service.getProcessedRecordForSnowpipe(record); assertEquals(testCase.expectedJson, mapper.readTree(got)); @@ -67,7 +66,8 @@ public static Iterable data() throws IOException { getAvro(), mapper.readTree( "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":\"string" - + " value\"}}"), null), + + " value\"}}"), + null), new Case( "string key, avro without registry value", getString(), @@ -75,68 +75,79 @@ public static Iterable data() throws IOException { mapper.readTree( "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" + " value\"}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}"), null), + + " value\"}}"), + null), new Case( "string key, json value", getString(), getJson(), mapper.readTree( "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}"), null), + + " value\"}}"), + null), new Case( "avro key, avro value", getAvro(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}"), null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}"), + null), new Case( "avro key, avro without registry value", getAvro(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}"), null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}"), + null), new Case( "avro key, json value", getAvro(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}"), null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}"), + null), new Case( "avro without registry key, avro value", getAvroWithoutRegistryKey(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}"), null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}"), + null), new Case( "avro without registry key, avro without registry value", getAvroWithoutRegistryKey(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}"), null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}"), + null), new Case( "avro without registry key, json value", getAvroWithoutRegistryKey(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}"), null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}"), + null), new Case( "json key, avro value", getJson(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}"), null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}"), + null), new Case( "json key, avro without registry value", getJson(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}"), null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}"), + null), new Case( "json key, json value", getJson(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}"), null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}"), + null), new Case( "multi line avro key, multi line avro value", getAvroMultiLine(), @@ -145,15 +156,15 @@ public static Iterable data() throws IOException { "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"topic\":\"test\",\"offset\":0,\"partition\":0,\"key\":[{\"username\":\"miguno\",\"tweet\":\"Rock:" + " Nerf paper, scissors is" + " fine.\",\"timestamp\":1366150681},{\"username\":\"BlizzardCS\",\"tweet\":\"Works" - + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}"), null), + + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}"), + null), new Case( "null key, json value", getNull(), getJson(), mapper.readTree( "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"), - SnowflakeErrors.ERROR_5016.getException()) - ); + SnowflakeErrors.ERROR_5016.getException())); } public static SchemaAndValue getString() { @@ -221,7 +232,12 @@ private static class Case { JsonNode expectedJson; Exception expectedException; - public Case(String name, SchemaAndValue key, SchemaAndValue value, JsonNode expectedJson, Exception expectedException) { + public Case( + String name, + SchemaAndValue key, + SchemaAndValue value, + JsonNode expectedJson, + Exception expectedException) { this.name = name; this.key = key; this.value = value; From 014b8ebc2e4142f5752a1d5a053549665242f15b Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 14:01:38 -0700 Subject: [PATCH 23/42] nonnull keys --- .../com/snowflake/kafka/connector/internal/SinkServiceIT.java | 4 ++-- .../internal/streaming/SnowflakeSinkServiceV2IT.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index 16b2043a7..06dd190e4 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -536,7 +536,7 @@ public void testNativeNullIngestion() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 2); + topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 2); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) @@ -724,7 +724,7 @@ public void testSinkServiceNegative() { SnowflakeConverter converter = new SnowflakeJsonConverter(); SchemaAndValue input = converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); - service.insert(new SinkRecord(topic, partition, null, null, input.schema(), input.value(), 0)); + service.insert(new SinkRecord(topic, partition, Schema.STRING_SCHEMA, "key", input.schema(), input.value(), 0)); service.startPartition(table, new TopicPartition(topic, partition)); } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 2e4d6163b..256a41b9b 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -904,7 +904,7 @@ public void testBrokenRecordIngestionFollowedUpByValidRecord() throws Exception SinkRecord correctValue = new SinkRecord( - topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 2); + topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 2); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); @@ -952,7 +952,7 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 0); + topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 0); SinkRecord brokenValue = new SinkRecord( From 434ebc2934464ff554a50b6ac3a6dbfa456eb1c6 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 14:03:02 -0700 Subject: [PATCH 24/42] autoformatting --- .../kafka/connector/internal/SinkServiceIT.java | 12 ++++++++++-- .../streaming/SnowflakeSinkServiceV2IT.java | 16 ++++++++++++++-- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index 06dd190e4..c4a8fd931 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -536,7 +536,13 @@ public void testNativeNullIngestion() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 2); + topic, + partition, + Schema.STRING_SCHEMA, + "key", + correctInputValue.schema(), + correctInputValue.value(), + 2); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) @@ -724,7 +730,9 @@ public void testSinkServiceNegative() { SnowflakeConverter converter = new SnowflakeJsonConverter(); SchemaAndValue input = converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); - service.insert(new SinkRecord(topic, partition, Schema.STRING_SCHEMA, "key", input.schema(), input.value(), 0)); + service.insert( + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "key", input.schema(), input.value(), 0)); service.startPartition(table, new TopicPartition(topic, partition)); } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 256a41b9b..282afc916 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -904,7 +904,13 @@ public void testBrokenRecordIngestionFollowedUpByValidRecord() throws Exception SinkRecord correctValue = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 2); + topic, + partition, + Schema.STRING_SCHEMA, + "key", + correctInputValue.schema(), + correctInputValue.value(), + 2); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); @@ -952,7 +958,13 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 0); + topic, + partition, + Schema.STRING_SCHEMA, + "key", + correctInputValue.schema(), + correctInputValue.value(), + 0); SinkRecord brokenValue = new SinkRecord( From 6583657577201382a39591ab1a496190c639e3d2 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 16:04:10 -0700 Subject: [PATCH 25/42] fix it again --- .../connector/internal/streaming/SnowflakeSinkServiceV2IT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 282afc916..3b0433ab0 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -976,7 +976,7 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord anotherCorrectValue = new SinkRecord( - topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 3); + topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 3); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); From abc32d38ed8f1fecc495b01289065627c6b9c94e Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 16:04:14 -0700 Subject: [PATCH 26/42] autoformatting --- .../internal/streaming/SnowflakeSinkServiceV2IT.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 3b0433ab0..97d601168 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -976,7 +976,13 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord anotherCorrectValue = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "key", correctInputValue.schema(), correctInputValue.value(), 3); + topic, + partition, + Schema.STRING_SCHEMA, + "key", + correctInputValue.schema(), + correctInputValue.value(), + 3); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); From 4ba153e08c53d9fb661074fd320e16f80bb3c0e2 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 19:02:25 -0700 Subject: [PATCH 27/42] simplify --- .../kafka/connector/SnowflakeSinkTask.java | 13 +++ .../internal/SnowflakeSinkService.java | 7 ++ .../internal/SnowflakeSinkServiceFactory.java | 28 ++++++- .../internal/SnowflakeSinkServiceV1.java | 22 ++++- .../streaming/SnowflakeSinkServiceV2.java | 26 +++++- .../streaming/TopicPartitionChannel.java | 8 +- .../connector/records/RecordService.java | 82 +++++++++++-------- .../SnowflakeSinkTaskStreamingTest.java | 4 +- .../connector/internal/MetaColumnIT.java | 3 +- .../streaming/SnowflakeSinkServiceV2IT.java | 24 +----- .../streaming/TopicPartitionChannelTest.java | 4 +- .../connector/records/ProcessRecordTest.java | 67 +++++---------- .../connector/records/RecordContentTest.java | 62 ++------------ 13 files changed, 173 insertions(+), 177 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java index 1831cec31..335aebd22 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkTask.java @@ -172,6 +172,18 @@ public void start(final Map parsedConfig) { final long bufferFlushTime = Long.parseLong(parsedConfig.get(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC)); + // Falling back to default behavior which is to ingest an empty json string if we get null + // value. (Tombstone record) + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; + if (parsedConfig.containsKey(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)) { + // we can always assume here that value passed in would be an allowed value, otherwise the + // connector would never start or reach the sink task stage + behavior = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.valueOf( + parsedConfig.get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)); + } + // we would have already validated the config inside SFConnector start() boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; if (parsedConfig.containsKey(SnowflakeSinkConnectorConfig.JMX_OPT)) { @@ -209,6 +221,7 @@ public void start(final Map parsedConfig) { .setFlushTime(bufferFlushTime) .setTopic2TableMap(topic2table) .setMetadataConfig(metadataConfig) + .setBehaviorOnNullValuesConfig(behavior) .setCustomJMXMetrics(enableCustomJMXMonitoring) .setErrorReporter(kafkaRecordErrorReporter) .setSinkTaskContext(this.context) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java index 90c722067..6983d0572 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkService.java @@ -2,6 +2,7 @@ import com.codahale.metrics.MetricRegistry; import com.google.common.annotations.VisibleForTesting; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; import java.util.Collection; @@ -133,9 +134,15 @@ public interface SnowflakeSinkService { /** @return current file size limitation */ long getFileSize(); + /* Set the behavior on what action to perform when this( @see com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig#BEHAVIOR_ON_NULL_VALUES_CONFIG ) config is set. */ + void setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior); + /* Should we emit Custom SF JMX Metrics to Mbean Server? If true (Default), we emit in form of SimpleMbeans */ void setCustomJMXMetrics(boolean enableJMX); + /* Only used in testing and verifying what was the passed value of this behavior from config to sink service*/ + SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig(); + /* Set Error reporter which can be used to send records to DLQ (Dead Letter Queue) */ default void setErrorReporter(KafkaRecordErrorReporter kafkaRecordErrorReporter) {} diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java index 0e35556d0..e0eacf173 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceFactory.java @@ -1,10 +1,10 @@ package com.snowflake.kafka.connector.internal; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2; import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig; -import java.util.HashMap; import java.util.Map; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -23,8 +23,17 @@ public static SnowflakeSinkServiceBuilder builder( SnowflakeConnectionService conn, IngestionMethodConfig ingestionType, Map connectorConfig) { - return new SnowflakeSinkServiceBuilder( - conn, ingestionType, connectorConfig == null ? new HashMap<>() : connectorConfig); + return new SnowflakeSinkServiceBuilder(conn, ingestionType, connectorConfig); + } + + /** + * Basic builder which internally uses SinkServiceV1 (Snowpipe) + * + * @param conn connection instance for connecting to snowflake + * @return A wrapper(Builder) having SinkService instance + */ + public static SnowflakeSinkServiceBuilder builder(SnowflakeConnectionService conn) { + return new SnowflakeSinkServiceBuilder(conn); } /** Builder class to create instance of {@link SnowflakeSinkService} */ @@ -38,7 +47,7 @@ private SnowflakeSinkServiceBuilder( IngestionMethodConfig ingestionType, Map connectorConfig) { if (ingestionType == IngestionMethodConfig.SNOWPIPE) { - this.service = new SnowflakeSinkServiceV1(conn, connectorConfig); + this.service = new SnowflakeSinkServiceV1(conn); } else { this.service = new SnowflakeSinkServiceV2(conn, connectorConfig); } @@ -46,6 +55,10 @@ private SnowflakeSinkServiceBuilder( LOGGER.info("{} created", this.service.getClass().getName()); } + private SnowflakeSinkServiceBuilder(SnowflakeConnectionService conn) { + this(conn, IngestionMethodConfig.SNOWPIPE, null /* Not required for V1 */); + } + /** * Add task for table and TopicPartition. Mostly used only for testing. When connector starts, * startTask is directly called. @@ -98,6 +111,13 @@ public SnowflakeSinkServiceBuilder setMetadataConfig(SnowflakeMetadataConfig con return this; } + public SnowflakeSinkServiceBuilder setBehaviorOnNullValuesConfig( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { + this.service.setBehaviorOnNullValuesConfig(behavior); + LOGGER.info("Config Behavior on null value is {}", behavior.toString()); + return this; + } + public SnowflakeSinkServiceBuilder setCustomJMXMetrics(final boolean enableJMX) { this.service.setCustomJMXMetrics(enableJMX); LOGGER.info("Config JMX value {}. (true = Enabled, false = Disabled)", enableJMX); diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index 74f36089d..f0ec946aa 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -69,11 +69,14 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { private final SnowflakeTelemetryService telemetryService; private Map topic2TableMap; + // Behavior to be set at the start of connector start. (For tombstone records) + private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues; + // default is true unless the configuration provided is false; // If this is true, we will enable Mbean for required classes and emit JMX metrics for monitoring private boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; - SnowflakeSinkServiceV1(SnowflakeConnectionService conn, Map connectorConfig) { + SnowflakeSinkServiceV1(SnowflakeConnectionService conn) { if (conn == null || conn.isClosed()) { throw SnowflakeErrors.ERROR_5010.getException(); } @@ -85,8 +88,12 @@ class SnowflakeSinkServiceV1 implements SnowflakeSinkService { this.conn = conn; isStopped = false; this.telemetryService = conn.getTelemetryClient(); - this.recordService = new RecordService(connectorConfig); + this.recordService = new RecordService(this.telemetryService); this.topic2TableMap = new HashMap<>(); + + // Setting the default value in constructor + // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) + this.behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; } /** @@ -296,11 +303,22 @@ public long getFileSize() { return this.fileSize; } + @Override + public void setBehaviorOnNullValuesConfig( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { + this.behaviorOnNullValues = behavior; + } + @Override public void setCustomJMXMetrics(boolean enableJMX) { this.enableCustomJMXMonitoring = enableJMX; } + @Override + public SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig() { + return this.behaviorOnNullValues; + } + /** * Loop through all pipes in memory and find out the metric registry instance for that pipe. The * pipes object's key is not pipeName hence need to loop over. diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index de47b395a..e4a8997b2 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -63,6 +63,9 @@ public class SnowflakeSinkServiceV2 implements SnowflakeSinkService { private final SnowflakeTelemetryService telemetryService; private Map topicToTableMap; + // Behavior to be set at the start of connector start. (For tombstone records) + private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues; + // default is true unless the configuration provided is false; // If this is true, we will enable Mbean for required classes and emit JMX metrics for monitoring private boolean enableCustomJMXMonitoring = SnowflakeSinkConnectorConfig.JMX_OPT_DEFAULT; @@ -106,14 +109,17 @@ public SnowflakeSinkServiceV2( this.flushTimeSeconds = StreamingUtils.STREAMING_BUFFER_FLUSH_TIME_DEFAULT_SEC; this.conn = conn; this.telemetryService = conn.getTelemetryClient(); - this.recordService = new RecordService(connectorConfig); + this.recordService = new RecordService(this.telemetryService); this.topicToTableMap = new HashMap<>(); + + // Setting the default value in constructor + // meaning it will not ignore the null values (Tombstone records wont be ignored/filtered) + this.behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; + this.connectorConfig = connectorConfig; this.enableSchematization = - Boolean.parseBoolean( - connectorConfig.getOrDefault( - SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + this.recordService.setAndGetEnableSchematizationFromConfig(this.connectorConfig); this.streamingIngestClient = StreamingClientProvider.getStreamingClientProviderInstance() @@ -148,6 +154,7 @@ public SnowflakeSinkServiceV2( this.recordService = recordService; this.telemetryService = telemetryService; this.topicToTableMap = topicToTableMap; + this.behaviorOnNullValues = behaviorOnNullValues; this.enableCustomJMXMonitoring = enableCustomJMXMonitoring; this.kafkaRecordErrorReporter = kafkaRecordErrorReporter; this.sinkTaskContext = sinkTaskContext; @@ -456,11 +463,22 @@ public long getFileSize() { return this.fileSizeBytes; } + @Override + public void setBehaviorOnNullValuesConfig( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { + this.behaviorOnNullValues = behavior; + } + @Override public void setCustomJMXMetrics(boolean enableJMX) { this.enableCustomJMXMonitoring = enableJMX; } + @Override + public SnowflakeSinkConnectorConfig.BehaviorOnNullValues getBehaviorOnNullValuesConfig() { + return this.behaviorOnNullValues; + } + /* Set this to send records to DLQ. */ @Override public void setErrorReporter(KafkaRecordErrorReporter kafkaRecordErrorReporter) { diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java index 91d6c0aca..dbf0b8f22 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannel.java @@ -11,7 +11,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; import com.snowflake.kafka.connector.internal.BufferThreshold; @@ -200,7 +199,7 @@ public TopicPartitionChannel( kafkaRecordErrorReporter, sinkTaskContext, null, /* Null Connection */ - new RecordService(), + new RecordService(null /* Null Telemetry Service*/), null); } @@ -260,9 +259,8 @@ public TopicPartitionChannel( /* Schematization related properties */ this.enableSchematization = - Boolean.parseBoolean( - sfConnectorConfig.getOrDefault( - SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "false")); + this.recordService.setAndGetEnableSchematizationFromConfig(sfConnectorConfig); + this.enableSchemaEvolution = this.enableSchematization && hasSchemaEvolutionPermission; // Open channel and reset the offset in kafka diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 08fa3a4a0..3e97ae58d 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -34,6 +34,8 @@ import java.util.List; import java.util.Map; import java.util.TimeZone; + +import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonProcessingException; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.JsonNode; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; @@ -86,10 +88,11 @@ public class RecordService { // This class is designed to work with empty metadata config map private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); - private Map connectorConfig; + /** Send Telemetry Data to Snowflake */ + private final SnowflakeTelemetryService telemetryService; private boolean enableSchematization; - private boolean ingestTombstoneRecords = - true; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion + private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", @@ -97,36 +100,59 @@ public class RecordService { * *

create a JsonRecordService instance */ - public RecordService(Map connectorConfig) { - this.connectorConfig = connectorConfig; + public RecordService(SnowflakeTelemetryService telemetryService) { + this.telemetryService = telemetryService; + } + // TESTING ONLY - create empty record service + @VisibleForTesting + public RecordService() { + this(null); + } + + public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { + this.metadataConfig = metadataConfigIn; + } + /** + * extract enableSchematization from the connector config and set the value for the recordService + * + *

The extracted boolean is returned for external usage. + * + * @param connectorConfig the connector config map + * @return a boolean indicating whether schematization is enabled + */ + public boolean setAndGetEnableSchematizationFromConfig( + final Map connectorConfig) { if (connectorConfig.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) { this.enableSchematization = Boolean.parseBoolean( connectorConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)); } - if (connectorConfig.containsKey(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG)) { - this.ingestTombstoneRecords = - connectorConfig - .get(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG) - .equalsIgnoreCase( - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - } + return this.enableSchematization; } - // TESTING ONLY - inject schematization and tombstone behavior + /** + * Directly set the enableSchematization through param + * + *

This method is only for testing + * + * @param enableSchematization whether we should enable schematization or not + */ @VisibleForTesting - public RecordService(boolean enableSchematization, boolean ingestTombstoneRecords) { + public void setEnableSchematization(final boolean enableSchematization) { this.enableSchematization = enableSchematization; - this.ingestTombstoneRecords = ingestTombstoneRecords; } - // TESTING ONLY - create empty record service + /** + * Directly set the behaviorOnNullValues through param + * + *

This method is only for testing + * + * @param behaviorOnNullValues how to handle null values + */ @VisibleForTesting - public RecordService() {} - - public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { - this.metadataConfig = metadataConfigIn; + public void setBehaviorOnNullValues(final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues) { + this.behaviorOnNullValues = behaviorOnNullValues; } /** @@ -144,7 +170,7 @@ private SnowflakeTableRow processRecord(SinkRecord record) { } if (record.value() == null || record.valueSchema() == null) { - if (this.ingestTombstoneRecords) { + if (this.behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { valueContent = new SnowflakeRecordContent(); } else { throw SnowflakeErrors.ERROR_5023.getException(); @@ -516,7 +542,7 @@ else if (value instanceof ByteBuffer) { * null case */ public boolean shouldSkipNullValue(SinkRecord record) { - if (this.ingestTombstoneRecords) { + if (this.behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { return false; } else { boolean isRecordValueNull = false; @@ -562,16 +588,4 @@ public boolean shouldSkipNullValue(SinkRecord record) { } return false; } - - // TESTING ONLY - @VisibleForTesting - public boolean getEnableSchematization() { - return this.enableSchematization; - } - - // TESTING ONLY - @VisibleForTesting - public boolean getIngestTombstoneRecords() { - return this.ingestTombstoneRecords; - } } diff --git a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java index 131dfdc4e..3e8678645 100644 --- a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java +++ b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskStreamingTest.java @@ -126,8 +126,8 @@ public void testSinkTaskInvalidRecord_InMemoryDLQ() throws Exception { new SinkRecord( topicName, partition, - Schema.STRING_SCHEMA, - "String", + null, + null, brokenInputValue.schema(), brokenInputValue.value(), 0); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java b/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java index 66b708937..bbb0a3478 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/MetaColumnIT.java @@ -1,7 +1,6 @@ package com.snowflake.kafka.connector.internal; import com.snowflake.kafka.connector.Utils; -import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import java.nio.charset.StandardCharsets; @@ -37,7 +36,7 @@ public void testKey() throws Exception { conn.createStage(stageName); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(3) .build(); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 97d601168..2e4d6163b 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -904,13 +904,7 @@ public void testBrokenRecordIngestionFollowedUpByValidRecord() throws Exception SinkRecord correctValue = new SinkRecord( - topic, - partition, - Schema.STRING_SCHEMA, - "key", - correctInputValue.schema(), - correctInputValue.value(), - 2); + topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 2); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); @@ -958,13 +952,7 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, - partition, - Schema.STRING_SCHEMA, - "key", - correctInputValue.schema(), - correctInputValue.value(), - 0); + topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 0); SinkRecord brokenValue = new SinkRecord( @@ -976,13 +964,7 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { SinkRecord anotherCorrectValue = new SinkRecord( - topic, - partition, - Schema.STRING_SCHEMA, - "key", - correctInputValue.schema(), - correctInputValue.value(), - 3); + topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 3); InMemoryKafkaRecordErrorReporter errorReporter = new InMemoryKafkaRecordErrorReporter(); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java index 322d9e46a..1c20b0798 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java @@ -224,7 +224,7 @@ public void testCloseChannelException() throws Exception { mockKafkaRecordErrorReporter, mockSinkTaskContext, mockSnowflakeConnectionService, - new RecordService(), + new RecordService(mockTelemetryService), mockTelemetryService); topicPartitionChannel.closeChannel(); @@ -614,7 +614,7 @@ public void testInsertRows_ValidationResponseHasErrors_NoErrorTolerance() throws mockKafkaRecordErrorReporter, mockSinkTaskContext, mockSnowflakeConnectionService, - new RecordService(), + new RecordService(mockTelemetryService), mockTelemetryService); List records = TestUtils.createJsonStringSinkRecords(0, 1, TOPIC, PARTITION); diff --git a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java index e04f65e29..7482f7af8 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java @@ -2,7 +2,6 @@ import static org.junit.Assert.assertEquals; -import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.mock.MockSchemaRegistryClient; import java.io.IOException; import java.net.URL; @@ -49,12 +48,9 @@ public void test() throws IOException { testCase.value.value(), partition); - try { - String got = service.getProcessedRecordForSnowpipe(record); - assertEquals(testCase.expectedJson, mapper.readTree(got)); - } catch (Exception ex) { - assert testCase.expectedException.getMessage().equals(ex.getMessage()); - } + String got = service.getProcessedRecordForSnowpipe(record); + + assertEquals(testCase.expected, mapper.readTree(got)); } @Parameterized.Parameters(name = "{index}: {0}") @@ -66,8 +62,7 @@ public static Iterable data() throws IOException { getAvro(), mapper.readTree( "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":\"string" - + " value\"}}"), - null), + + " value\"}}")), new Case( "string key, avro without registry value", getString(), @@ -75,79 +70,68 @@ public static Iterable data() throws IOException { mapper.readTree( "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" + " value\"}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}"), - null), + + " value\"}}")), new Case( "string key, json value", getString(), getJson(), mapper.readTree( "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":\"string" - + " value\"}}"), - null), + + " value\"}}")), new Case( "avro key, avro value", getAvro(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}"), - null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"int\":222},\"key_schema_id\":1}}")), new Case( "avro key, avro without registry value", getAvro(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}"), - null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"int\":222}],\"key_schema_id\":1}}")), new Case( "avro key, json value", getAvro(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}"), - null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"int\":222},\"key_schema_id\":1}}")), new Case( "avro without registry key, avro value", getAvroWithoutRegistryKey(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}"), - null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"id\":\"aabbccdd\"}}}")), new Case( "avro without registry key, avro without registry value", getAvroWithoutRegistryKey(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}"), - null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"id\":\"aabbccdd\"}]}}")), new Case( "avro without registry key, json value", getAvroWithoutRegistryKey(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}"), - null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"id\":\"aabbccdd\"}}}")), new Case( "json key, avro value", getJson(), getAvro(), mapper.readTree( - "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}"), - null), + "{\"content\":{\"int\":222},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"schema_id\":1,\"key\":{\"some_field\":\"some_value\"}}}")), new Case( "json key, avro without registry value", getJson(), getAvroWithoutRegistryValue(), mapper.readTree( - "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}"), - null), + "{\"content\":{\"name\":\"foo\",\"age\":30},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}{\"content\":{\"name\":\"bar\",\"age\":29},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":[{\"some_field\":\"some_value\"}]}}")), new Case( "json key, json value", getJson(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}"), - null), + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0,\"key\":{\"some_field\":\"some_value\"}}}")), new Case( "multi line avro key, multi line avro value", getAvroMultiLine(), @@ -156,15 +140,13 @@ public static Iterable data() throws IOException { "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"topic\":\"test\",\"offset\":0,\"partition\":0,\"key\":[{\"username\":\"miguno\",\"tweet\":\"Rock:" + " Nerf paper, scissors is" + " fine.\",\"timestamp\":1366150681},{\"username\":\"BlizzardCS\",\"tweet\":\"Works" - + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}"), - null), + + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}")), new Case( "null key, json value", getNull(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"), - SnowflakeErrors.ERROR_5016.getException())); + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"))); } public static SchemaAndValue getString() { @@ -229,20 +211,13 @@ private static class Case { String name; SchemaAndValue key; SchemaAndValue value; - JsonNode expectedJson; - Exception expectedException; - - public Case( - String name, - SchemaAndValue key, - SchemaAndValue value, - JsonNode expectedJson, - Exception expectedException) { + JsonNode expected; + + public Case(String name, SchemaAndValue key, SchemaAndValue value, JsonNode expected) { this.name = name; this.key = key; this.value = value; - this.expectedJson = expectedJson; - this.expectedException = expectedException; + this.expected = expected; } @Override diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index b337a1192..c66588669 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -128,7 +128,8 @@ public void test() throws IOException { @Test(expected = SnowflakeKafkaConnectorException.class) public void testEmptyValueDisabledTombstone() { - RecordService service = new RecordService(false, false); + RecordService service = new RecordService(); + service.setBehaviorOnNullValues(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE); SinkRecord record = new SinkRecord(topic, partition, null, null, Schema.STRING_SCHEMA, null, partition); @@ -139,7 +140,8 @@ public void testEmptyValueDisabledTombstone() { public void testEmptyValueSchemaDisabledTombstone() throws IOException { JsonNode data = mapper.readTree("{\"name\":123}"); SnowflakeRecordContent content = new SnowflakeRecordContent(data); - RecordService service = new RecordService(false, false); + RecordService service = new RecordService(); + service.setBehaviorOnNullValues(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE); SinkRecord record = new SinkRecord(topic, partition, null, null, null, content, partition); service.getProcessedRecordForSnowpipe(record); @@ -237,7 +239,7 @@ public void testConvertToJsonReadOnlyByteBuffer() { @Test public void testSchematizationStringField() throws JsonProcessingException { - RecordService service = new RecordService(true, false); + RecordService service = new RecordService(); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); String value = "{\"name\":\"sf\",\"answer\":42}"; @@ -258,7 +260,7 @@ public void testSchematizationStringField() throws JsonProcessingException { @Test public void testColumnNameFormatting() throws JsonProcessingException { - RecordService service = new RecordService(true, false); + RecordService service = new RecordService(); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); String value = "{\"\\\"NaMe\\\"\":\"sf\",\"AnSwEr\":42}"; @@ -274,38 +276,9 @@ public void testColumnNameFormatting() throws JsonProcessingException { assert got.containsKey("AnSwEr"); } - @Test - public void testRecordServiceConfigCreation() { - this.testRecordServiceConfigCreationRunner( - true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - this.testRecordServiceConfigCreationRunner( - false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - this.testRecordServiceConfigCreationRunner( - true, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - this.testRecordServiceConfigCreationRunner( - false, SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - } - - private void testRecordServiceConfigCreationRunner( - boolean enableSchematization, String behavior) { - Map config = new HashMap<>(); - - config.put( - SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, - String.valueOf(enableSchematization)); - config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, behavior); - - RecordService recordService = new RecordService(config); - - assert recordService.getEnableSchematization() == enableSchematization - && recordService.getIngestTombstoneRecords() - == (behavior.equals( - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString())); - } - @Test public void testGetProcessedRecordNullValue() throws JsonProcessingException { - RecordService service = new RecordService(false, true); + RecordService service = new RecordService(); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); SchemaAndValue sv = jsonConverter.toConnectData(topic, null); @@ -332,25 +305,4 @@ public void testGetProcessedRecordNullValue() throws JsonProcessingException { .count() == 1; } - - @Test(expected = SnowflakeKafkaConnectorException.class) - public void testGetProcessedRecordNullKeyFails() throws JsonProcessingException { - RecordService service = new RecordService(false, true); - SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); - - SchemaAndValue sv = jsonConverter.toConnectData(topic, null); - - SinkRecord allNullRecord = - new SinkRecord(topic, partition, null, null, sv.schema(), sv.value(), partition); - SinkRecord nullKeyRecord = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition); - SinkRecord nullKeySchemaRecord = - new SinkRecord(topic, partition, null, "string", sv.schema(), sv.value(), partition); - - // null key schema or key will fail - service.getProcessedRecordForStreamingIngest(allNullRecord); - service.getProcessedRecordForStreamingIngest(nullKeyRecord); - service.getProcessedRecordForStreamingIngest(nullKeySchemaRecord); - } } From 8673464baa2528062a7943963def63afca79005a Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 19:02:31 -0700 Subject: [PATCH 28/42] autoformatting --- .../kafka/connector/records/RecordService.java | 10 ++++++---- .../kafka/connector/records/RecordContentTest.java | 1 - 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 3e97ae58d..15c33106d 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -23,6 +23,7 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.internal.KCLogger; import com.snowflake.kafka.connector.internal.SnowflakeErrors; +import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.text.SimpleDateFormat; @@ -34,8 +35,6 @@ import java.util.List; import java.util.Map; import java.util.TimeZone; - -import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonProcessingException; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.JsonNode; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; @@ -90,9 +89,11 @@ public class RecordService { private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); /** Send Telemetry Data to Snowflake */ private final SnowflakeTelemetryService telemetryService; + private boolean enableSchematization; private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion + SnowflakeSinkConnectorConfig.BehaviorOnNullValues + .DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", @@ -151,7 +152,8 @@ public void setEnableSchematization(final boolean enableSchematization) { * @param behaviorOnNullValues how to handle null values */ @VisibleForTesting - public void setBehaviorOnNullValues(final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues) { + public void setBehaviorOnNullValues( + final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues) { this.behaviorOnNullValues = behaviorOnNullValues; } diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index c66588669..82dbbdc60 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -10,7 +10,6 @@ import java.util.Arrays; import java.util.Base64; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonProcessingException; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.type.TypeReference; From ffa955b6ab0bacf2bf1c96b1a2b9b2b798ad72b6 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 19:14:05 -0700 Subject: [PATCH 29/42] tests --- .../connector/internal/SnowflakeErrors.java | 12 ++-- .../internal/SnowflakeSinkServiceV1.java | 2 +- .../streaming/SnowflakeSinkServiceV2.java | 2 +- .../connector/records/RecordService.java | 30 +++++----- .../connector/internal/SinkServiceIT.java | 60 ++++++++----------- ...SnowflakeTelemetryPipeStatusMetricsIT.java | 5 +- 6 files changed, 47 insertions(+), 64 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index cbf47d018..3e1790779 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -284,7 +284,10 @@ public enum SnowflakeErrors { ERROR_5016( "5016", "Invalid SinkRecord received", - "SinkRecord.key and SinkRecord.keySchema cannot be null"), + "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" + + " is enabled (see " + + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + + " for more information."), ERROR_5017( "5017", "Invalid api call to cached put", "Cached put only support AWS, Azure and GCS."), ERROR_5018("5018", "Failed to execute cached put", "Error in cached put command"), @@ -295,13 +298,6 @@ public enum SnowflakeErrors { "Failed to get data schema", "Failed to get data schema. Unrecognizable data type in JSON object"), ERROR_5022("5022", "Invalid column name", "Failed to find column in the schema"), - ERROR_5023( - "5016", - "Invalid SinkRecord received", - "SinkRecord.value and SinkRecord.valueSchema cannot be null unless tombstone record ingestion" - + " is enabled (see " - + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG - + " for more information."), ; // properties diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java index f0ec946aa..530e8851a 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeSinkServiceV1.java @@ -130,7 +130,7 @@ public void insert(final Collection records) { // note that records can be empty for (SinkRecord record : records) { // check if need to handle null value records - if (recordService.shouldSkipNullValue(record)) { + if (recordService.shouldSkipNullValue(record, behaviorOnNullValues)) { continue; } // Might happen a count of record based flushing diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index e4a8997b2..884a91d29 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -259,7 +259,7 @@ public void insert(Collection records) { // note that records can be empty but, we will still need to check for time based flush for (SinkRecord record : records) { // check if need to handle null value records - if (recordService.shouldSkipNullValue(record)) { + if (recordService.shouldSkipNullValue(record, behaviorOnNullValues)) { continue; } // While inserting into buffer, we will check for count threshold and buffered bytes diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 15c33106d..90ae97042 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -70,6 +70,11 @@ public class RecordService { private static final String KEY_SCHEMA_ID = "key_schema_id"; static final String HEADERS = "headers"; + private boolean enableSchematization = false; + private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = + SnowflakeSinkConnectorConfig.BehaviorOnNullValues + .DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion + // For each task, we require a separate instance of SimpleDataFormat, since they are not // inherently thread safe static final ThreadLocal ISO_DATE_TIME_FORMAT = @@ -90,29 +95,26 @@ public class RecordService { /** Send Telemetry Data to Snowflake */ private final SnowflakeTelemetryService telemetryService; - private boolean enableSchematization; - private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues - .DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion - /** * process records output JSON format: { "meta": { "offset": 123, "topic": "topic name", * "partition": 123, "key":"key name" } "content": "record content" } * *

create a JsonRecordService instance + * + * @param telemetryService Telemetry Service Instance. Can be null. */ public RecordService(SnowflakeTelemetryService telemetryService) { this.telemetryService = telemetryService; } - // TESTING ONLY - create empty record service + /** Record service with null telemetry Service, only use it for testing. */ @VisibleForTesting public RecordService() { this(null); } public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { - this.metadataConfig = metadataConfigIn; + metadataConfig = metadataConfigIn; } /** * extract enableSchematization from the connector config and set the value for the recordService @@ -166,16 +168,11 @@ public void setBehaviorOnNullValues( private SnowflakeTableRow processRecord(SinkRecord record) { SnowflakeRecordContent valueContent; - // cannot ingest null key - if (record.key() == null || record.keySchema() == null) { - throw SnowflakeErrors.ERROR_5016.getException(); - } - if (record.value() == null || record.valueSchema() == null) { if (this.behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { valueContent = new SnowflakeRecordContent(); } else { - throw SnowflakeErrors.ERROR_5023.getException(); + throw SnowflakeErrors.ERROR_5016.getException(); } } else { if (!record.valueSchema().name().equals(SnowflakeJsonSchema.NAME)) { @@ -539,12 +536,15 @@ else if (value instanceof ByteBuffer) { *

If the value is an empty JSON node, we could assume the value passed was null. * * @param record record sent from Kafka to KC + * @param behaviorOnNullValues behavior passed inside KC * @return true if we would skip adding it to buffer * @see com.snowflake.kafka.connector.records.SnowflakeJsonConverter#toConnectData when bytes == * null case */ - public boolean shouldSkipNullValue(SinkRecord record) { - if (this.behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { + public boolean shouldSkipNullValue( + SinkRecord record, + final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues) { + if (behaviorOnNullValues == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT) { return false; } else { boolean isRecordValueNull = false; diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index c4a8fd931..fb5cf7af1 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -6,7 +6,6 @@ import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; -import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import io.confluent.connect.avro.AvroConverter; @@ -14,8 +13,10 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.sql.ResultSet; import java.util.*; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; +import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -24,6 +25,7 @@ import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.After; +import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; @@ -49,8 +51,7 @@ public void afterEach() { @Test public void testSinkServiceBuilder() { // default value - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); + SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn).build(); assert service.getFileSize() == SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT; assert service.getFlushTime() == SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_DEFAULT; @@ -58,7 +59,7 @@ public void testSinkServiceBuilder() { // set some value service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setFileSize(SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT * 4) .setFlushTime(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN + 10) .setRecordNumber(10) @@ -70,7 +71,7 @@ public void testSinkServiceBuilder() { // set some invalid value service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(-100) .setFlushTime(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN - 10) .setFileSize(SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MIN - 1) @@ -82,16 +83,13 @@ public void testSinkServiceBuilder() { // connection test assert TestUtils.assertError( - SnowflakeErrors.ERROR_5010, - () -> - SnowflakeSinkServiceFactory.builder(null, IngestionMethodConfig.SNOWPIPE, null) - .build()); + SnowflakeErrors.ERROR_5010, () -> SnowflakeSinkServiceFactory.builder(null).build()); assert TestUtils.assertError( SnowflakeErrors.ERROR_5010, () -> { SnowflakeConnectionService conn = TestUtils.getConnectionService(); conn.close(); - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null).build(); + SnowflakeSinkServiceFactory.builder(conn).build(); }); } @@ -100,7 +98,7 @@ public void testIngestion() throws Exception { conn.createTable(table); conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -228,7 +226,7 @@ public void testNativeJsonInputIngestion() throws Exception { startOffset + 3); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -416,7 +414,7 @@ public void testNativeAvroInputIngestion() throws Exception { conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -499,7 +497,7 @@ public void testNativeBrokenIngestion() throws Exception { startOffset + 2); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -536,16 +534,10 @@ public void testNativeNullIngestion() throws Exception { SinkRecord correctValue = new SinkRecord( - topic, - partition, - Schema.STRING_SCHEMA, - "key", - correctInputValue.schema(), - correctInputValue.value(), - 2); + topic, partition, null, null, correctInputValue.schema(), correctInputValue.value(), 2); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -597,7 +589,7 @@ public void testNativeNullValueIngestion() throws Exception { SinkRecord brokenValue = new SinkRecord(topic, partition, null, null, null, null, 0); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(recordCount) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -614,7 +606,7 @@ public void testRecordNumber() throws Exception { int numLimit = 100; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setRecordNumber(numLimit) .setFlushTime(30) .addTask(table, new TopicPartition(topic, partition)) @@ -661,7 +653,7 @@ public void testFileSize() throws Exception { long size = 10000; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setFileSize(size) .setFlushTime(10) .addTask(table, new TopicPartition(topic, partition)) @@ -693,7 +685,7 @@ public void testFlushTime() throws Exception { long flushTime = 20; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setFlushTime(flushTime) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -718,9 +710,7 @@ public void testSinkServiceNegative() { conn.createTable(table); conn.createStage(stage); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) - .setRecordNumber(1) - .build(); + SnowflakeSinkServiceFactory.builder(conn).setRecordNumber(1).build(); TopicPartition topicPartition = new TopicPartition(topic, partition); service.getOffset(topicPartition); List topicPartitionList = new ArrayList<>(); @@ -730,9 +720,7 @@ public void testSinkServiceNegative() { SnowflakeConverter converter = new SnowflakeJsonConverter(); SchemaAndValue input = converter.toConnectData(topic, "{\"name\":\"test\"}".getBytes(StandardCharsets.UTF_8)); - service.insert( - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "key", input.schema(), input.value(), 0)); + service.insert(new SinkRecord(topic, partition, null, null, input.schema(), input.value(), 0)); service.startPartition(table, new TopicPartition(topic, partition)); } @@ -770,7 +758,7 @@ public void testRecoverReprocessFiles() throws Exception { assert getStageSize(stage, table, 0) == 4; SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .addTask(table, new TopicPartition(topic, partition)) .setRecordNumber(1) // immediate flush .build(); @@ -819,7 +807,7 @@ public void testBrokenRecord() { offset); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -852,7 +840,7 @@ public void testCleanerRecover() throws Exception { SnowflakeConnectionService spyConn = spy(conn); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(spyConn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(spyConn) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); @@ -919,7 +907,7 @@ public void testCleanerRecoverListCount() throws Exception { SnowflakeConnectionService spyConn = spy(conn); SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(spyConn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(spyConn) .setRecordNumber(1) .addTask(table, new TopicPartition(topic, partition)) .build(); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java index ce49639e3..69fc4fa30 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SnowflakeTelemetryPipeStatusMetricsIT.java @@ -6,7 +6,6 @@ import com.codahale.metrics.MetricRegistry; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.internal.metrics.MetricsUtil; -import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.records.SnowflakeConverter; import com.snowflake.kafka.connector.records.SnowflakeJsonConverter; import java.nio.charset.StandardCharsets; @@ -44,7 +43,7 @@ public void testJMXMetricsInMBeanServer() throws Exception { // This means that default is true. SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(1) .build(); @@ -182,7 +181,7 @@ public void testJMXDisabledInMBeanServer() { // This means that default is true. SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE, null) + SnowflakeSinkServiceFactory.builder(conn) .setCustomJMXMetrics(false) .addTask(tableName, new TopicPartition(topic, partition)) .setRecordNumber(1) From a6f76d74af368b0f7cd6cdfcab48c46ed8d71b0a Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 19:14:10 -0700 Subject: [PATCH 30/42] autoformatting --- .../com/snowflake/kafka/connector/records/RecordService.java | 2 +- .../com/snowflake/kafka/connector/internal/SinkServiceIT.java | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 90ae97042..76c5bc989 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -536,7 +536,7 @@ else if (value instanceof ByteBuffer) { *

If the value is an empty JSON node, we could assume the value passed was null. * * @param record record sent from Kafka to KC - * @param behaviorOnNullValues behavior passed inside KC + * @param behaviorOnNullValues behavior passed inside KC * @return true if we would skip adding it to buffer * @see com.snowflake.kafka.connector.records.SnowflakeJsonConverter#toConnectData when bytes == * null case diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index fb5cf7af1..db78eac9b 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -13,10 +13,8 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.sql.ResultSet; import java.util.*; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; -import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -25,7 +23,6 @@ import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.After; -import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; From 07e0db456520d08b2b4917fc768c35b9637b574d Mon Sep 17 00:00:00 2001 From: revi cheng Date: Tue, 22 Aug 2023 19:17:46 -0700 Subject: [PATCH 31/42] add schema enable back --- .../snowflake/kafka/connector/records/RecordContentTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 82dbbdc60..24f60d4d9 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -241,6 +241,7 @@ public void testSchematizationStringField() throws JsonProcessingException { RecordService service = new RecordService(); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); + service.setEnableSchematization(true); String value = "{\"name\":\"sf\",\"answer\":42}"; byte[] valueContents = (value).getBytes(StandardCharsets.UTF_8); SchemaAndValue sv = jsonConverter.toConnectData(topic, valueContents); @@ -262,6 +263,7 @@ public void testColumnNameFormatting() throws JsonProcessingException { RecordService service = new RecordService(); SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); + service.setEnableSchematization(true); String value = "{\"\\\"NaMe\\\"\":\"sf\",\"AnSwEr\":42}"; byte[] valueContents = (value).getBytes(StandardCharsets.UTF_8); SchemaAndValue sv = jsonConverter.toConnectData(topic, valueContents); From e644e902b6b91df0abba5540dc0f2af825933291 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Wed, 23 Aug 2023 11:08:31 -0700 Subject: [PATCH 32/42] add all null tests --- .../internal/TombstoneRecordIngestionIT.java | 44 +++++++++++++++++++ .../connector/records/ProcessRecordTest.java | 15 ++++++- 2 files changed, 58 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 8bc59fbb6..462108031 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -164,6 +164,50 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { service.closeAll(); } + @Test + public void testDefaultAllNullTombstoneRecordBehavior() throws Exception { + Map connectorConfig = TestUtils.getConfig(); + + if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { + conn.createTable(table); + conn.createStage(stage); + } else { + connectorConfig = TestUtils.getConfForStreaming(); + } + + // set default behavior + connectorConfig.put( + SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); + + TopicPartition topicPartition = new TopicPartition(topic, partition); + SnowflakeSinkService service = + SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) + .setRecordNumber(1) + .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) + .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) + .addTask(table, topicPartition) + .build(); + + // make tombstone record + SchemaAndValue input = converter.toConnectData(topic, null); + long offset = 0; + SinkRecord record1 = + new SinkRecord( + topic, partition, null, null, input.schema(), input.value(), offset); + + // test insert + service.insert(Collections.singletonList(record1)); + service.callAllGetOffset(); + + // verify inserted + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + TestUtils.assertWithRetry( + () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); + + service.closeAll(); + } + @Test public void testIgnoreTombstoneRecordBehavior() throws Exception { Map connectorConfig = TestUtils.getConfig(); diff --git a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java index 7482f7af8..1cd584aee 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java @@ -141,12 +141,25 @@ public static Iterable data() throws IOException { + " Nerf paper, scissors is" + " fine.\",\"timestamp\":1366150681},{\"username\":\"BlizzardCS\",\"tweet\":\"Works" + " as intended. Terran is IMBA.\",\"timestamp\":1366154481}]}}")), + new Case( + "json key, null value", + getJson(), + getNull(), + mapper.readTree( + "{\"content\":{},\"meta\":{\"topic\":\"test\",\"offset\":0,\"partition\":0,\"schema_id\":0,\"key\":{\"some_field\":\"some_value\"}}}")), new Case( "null key, json value", getNull(), getJson(), mapper.readTree( - "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}"))); + "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}")), + + new Case( + "null key, null value", + getNull(), + getNull(), + mapper.readTree( + "{\"content\":{},\"meta\":{\"topic\":\"test\",\"offset\":0,\"partition\":0,\"schema_id\":0}}"))); } public static SchemaAndValue getString() { From 27b66411c5b8d8ba9d2a35f5be134cd17070c754 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Wed, 23 Aug 2023 11:08:35 -0700 Subject: [PATCH 33/42] autoformatting --- .../kafka/connector/internal/TombstoneRecordIngestionIT.java | 3 +-- .../snowflake/kafka/connector/records/ProcessRecordTest.java | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 462108031..52ae42b40 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -193,8 +193,7 @@ public void testDefaultAllNullTombstoneRecordBehavior() throws Exception { SchemaAndValue input = converter.toConnectData(topic, null); long offset = 0; SinkRecord record1 = - new SinkRecord( - topic, partition, null, null, input.schema(), input.value(), offset); + new SinkRecord(topic, partition, null, null, input.schema(), input.value(), offset); // test insert service.insert(Collections.singletonList(record1)); diff --git a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java index 1cd584aee..38c86194c 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/ProcessRecordTest.java @@ -153,7 +153,6 @@ public static Iterable data() throws IOException { getJson(), mapper.readTree( "{\"content\":{\"some_field\":\"some_value\"},\"meta\":{\"offset\":0,\"topic\":\"test\",\"partition\":0}}")), - new Case( "null key, null value", getNull(), From 631f75a2c0bc5909993b7edb51ac4c9b2117eed4 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Wed, 23 Aug 2023 13:50:46 -0700 Subject: [PATCH 34/42] add null key test to streaming --- .../connector/internal/SnowflakeErrors.java | 3 +- .../connector/records/RecordService.java | 4 +- .../connector/records/RecordContentTest.java | 50 +++++++++++-------- 3 files changed, 31 insertions(+), 26 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java index 3e1790779..e3eadc238 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeErrors.java @@ -297,8 +297,7 @@ public enum SnowflakeErrors { "5021", "Failed to get data schema", "Failed to get data schema. Unrecognizable data type in JSON object"), - ERROR_5022("5022", "Invalid column name", "Failed to find column in the schema"), - ; + ERROR_5022("5022", "Invalid column name", "Failed to find column in the schema"); // properties diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 76c5bc989..6b5570bb3 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -73,7 +73,7 @@ public class RecordService { private boolean enableSchematization = false; private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = SnowflakeSinkConnectorConfig.BehaviorOnNullValues - .DEFAULT; // since BEHAVIOR_ON_NULL_VALUES_CONFIG defaults to ingestion + .DEFAULT; // For each task, we require a separate instance of SimpleDataFormat, since they are not // inherently thread safe @@ -317,7 +317,7 @@ public SnowflakeTableRow(SnowflakeRecordContent content, JsonNode metadata) { } void putKey(SinkRecord record, ObjectNode meta) { - if (record.key() == null) { + if (record.key() == null || record.keySchema() == null) { return; } diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 24f60d4d9..ab9182036 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -278,32 +278,38 @@ public void testColumnNameFormatting() throws JsonProcessingException { } @Test - public void testGetProcessedRecordNullValue() throws JsonProcessingException { - RecordService service = new RecordService(); + public void testGetProcessedRecord() throws JsonProcessingException { SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); - SchemaAndValue sv = jsonConverter.toConnectData(topic, null); String keyStr = "string"; - SinkRecord allNullRecord = - new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, null, partition); - SinkRecord nullValueRecord = + // all null + this.testGetProcessedRecordRunner( + new SinkRecord(topic, partition, null, null, null, null, partition), "{}", ""); + + // null value + this.testGetProcessedRecordRunner( + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition), "{}", keyStr); + this.testGetProcessedRecordRunner( + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition), "{}", keyStr); + + // null key + this.testGetProcessedRecordRunner( + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition), "{}", ""); + this.testGetProcessedRecordRunner( new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition); - SinkRecord nullValueSchemaRecord = - new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition); - - assert service.getProcessedRecordForStreamingIngest(allNullRecord).values().stream() - .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) - .count() - == 1; - assert service.getProcessedRecordForStreamingIngest(nullValueRecord).values().stream() - .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) - .count() - == 1; - assert service.getProcessedRecordForStreamingIngest(nullValueSchemaRecord).values().stream() - .filter(value -> value.toString().contains(topic) && value.toString().contains(keyStr)) - .count() - == 1; + topic, partition, null, keyStr, sv.schema(), sv.value(), partition), "{}", ""); + } + + private void testGetProcessedRecordRunner(SinkRecord record, String expectedRecordContent, String expectedRecordMetadataKey) throws JsonProcessingException { + RecordService service = new RecordService(); + Map recordData = service.getProcessedRecordForStreamingIngest(record); + + assert recordData.size() == 2; + assert recordData.get("RECORD_CONTENT").equals(expectedRecordContent); + assert recordData.get("RECORD_METADATA").toString().contains(expectedRecordMetadataKey); } } From c815b48cd97f5d04aaf659efdf4d33b1ee62c447 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Wed, 23 Aug 2023 13:50:50 -0700 Subject: [PATCH 35/42] autoformatting --- .../connector/records/RecordService.java | 3 +-- .../connector/records/RecordContentTest.java | 22 +++++++++++++------ 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 6b5570bb3..890697ef8 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -72,8 +72,7 @@ public class RecordService { private boolean enableSchematization = false; private SnowflakeSinkConnectorConfig.BehaviorOnNullValues behaviorOnNullValues = - SnowflakeSinkConnectorConfig.BehaviorOnNullValues - .DEFAULT; + SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT; // For each task, we require a separate instance of SimpleDataFormat, since they are not // inherently thread safe diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index ab9182036..f8df8b13c 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -290,21 +290,29 @@ public void testGetProcessedRecord() throws JsonProcessingException { // null value this.testGetProcessedRecordRunner( new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition), "{}", keyStr); + topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition), + "{}", + keyStr); this.testGetProcessedRecordRunner( - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition), "{}", keyStr); + new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition), + "{}", + keyStr); // null key this.testGetProcessedRecordRunner( new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition), "{}", ""); + topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition), + "{}", + ""); this.testGetProcessedRecordRunner( - new SinkRecord( - topic, partition, null, keyStr, sv.schema(), sv.value(), partition), "{}", ""); + new SinkRecord(topic, partition, null, keyStr, sv.schema(), sv.value(), partition), + "{}", + ""); } - private void testGetProcessedRecordRunner(SinkRecord record, String expectedRecordContent, String expectedRecordMetadataKey) throws JsonProcessingException { + private void testGetProcessedRecordRunner( + SinkRecord record, String expectedRecordContent, String expectedRecordMetadataKey) + throws JsonProcessingException { RecordService service = new RecordService(); Map recordData = service.getProcessedRecordForStreamingIngest(record); From eef26429788b401faf14831b884499061cec7444 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Wed, 23 Aug 2023 18:50:07 -0700 Subject: [PATCH 36/42] update test to call set not config --- .../connector/internal/TombstoneRecordIngestionIT.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 52ae42b40..d986b25a6 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -218,11 +218,6 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { connectorConfig = TestUtils.getConfForStreaming(); } - // set default behavior - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); - TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) @@ -232,6 +227,9 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { .addTask(table, topicPartition) .build(); + // set ignore behavior + service.setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE); + // make tombstone record SchemaAndValue record1Input = converter.toConnectData(topic, null); long record1Offset = 0; From 06c69baa1ea2dc7644d0614509a8a4b4f64b7f02 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 16:24:51 -0700 Subject: [PATCH 37/42] group it test --- .../internal/TombstoneRecordIngestionIT.java | 77 +++++-------------- 1 file changed, 19 insertions(+), 58 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index d986b25a6..5019ccc78 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -131,11 +131,6 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { connectorConfig = TestUtils.getConfForStreaming(); } - // set default behavior - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) @@ -145,64 +140,32 @@ public void testDefaultTombstoneRecordBehavior() throws Exception { .addTask(table, topicPartition) .build(); - // make tombstone record + // make tombstone records SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - SinkRecord record1 = + SinkRecord tombstoneRecord1 = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "test", input.schema(), input.value(), offset); - - // test insert - service.insert(Collections.singletonList(record1)); - service.callAllGetOffset(); - - // verify inserted - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); - TestUtils.assertWithRetry( - () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); - - service.closeAll(); - } - - @Test - public void testDefaultAllNullTombstoneRecordBehavior() throws Exception { - Map connectorConfig = TestUtils.getConfig(); - - if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { - conn.createTable(table); - conn.createStage(stage); - } else { - connectorConfig = TestUtils.getConfForStreaming(); - } - - // set default behavior - connectorConfig.put( - SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT.toString()); - - TopicPartition topicPartition = new TopicPartition(topic, partition); - SnowflakeSinkService service = - SnowflakeSinkServiceFactory.builder(conn, this.ingestionMethod, connectorConfig) - .setRecordNumber(1) - .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) - .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) - .addTask(table, topicPartition) - .build(); - - // make tombstone record - SchemaAndValue input = converter.toConnectData(topic, null); - long offset = 0; - SinkRecord record1 = - new SinkRecord(topic, partition, null, null, input.schema(), input.value(), offset); + topic, partition, Schema.STRING_SCHEMA, "tombstoneRecord1", input.schema(), input.value(), 0); + SinkRecord tombstoneRecord2 = + new SinkRecord( + topic, partition, Schema.STRING_SCHEMA, "tombstoneRecord2", null, null, 1); + SinkRecord allNullRecord1 = + new SinkRecord( + topic, partition, null, null, null, null, 2); + SinkRecord allNullRecord2 = + new SinkRecord( + topic, partition, null, null, input.schema(), input.value(), 3); + SinkRecord allNullRecord3 = + new SinkRecord( + topic, partition, input.schema(), input.value(), input.schema(), input.value(), 4); // test insert - service.insert(Collections.singletonList(record1)); + service.insert(Arrays.asList(tombstoneRecord1, tombstoneRecord2, allNullRecord1, allNullRecord2, allNullRecord3)); service.callAllGetOffset(); // verify inserted - TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 1, 30, 20); + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == 5, 30, 20); TestUtils.assertWithRetry( - () -> service.getOffset(new TopicPartition(topic, partition)) == 1, 20, 5); + () -> service.getOffset(new TopicPartition(topic, partition)) == 5, 20, 5); service.closeAll(); } @@ -225,11 +188,9 @@ public void testIgnoreTombstoneRecordBehavior() throws Exception { .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) .addTask(table, topicPartition) + .setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE) .build(); - // set ignore behavior - service.setBehaviorOnNullValuesConfig(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.IGNORE); - // make tombstone record SchemaAndValue record1Input = converter.toConnectData(topic, null); long record1Offset = 0; From abeaf39a03939690671a24ada8fef7e766685255 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 16:37:11 -0700 Subject: [PATCH 38/42] nits --- .../kafka/connector/records/RecordService.java | 4 +++- .../kafka/connector/internal/SinkServiceIT.java | 10 +++++++--- .../connector/internal/TombstoneRecordIngestionIT.java | 2 +- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 890697ef8..6557a8037 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -91,6 +91,7 @@ public class RecordService { // This class is designed to work with empty metadata config map private SnowflakeMetadataConfig metadataConfig = new SnowflakeMetadataConfig(); + /** Send Telemetry Data to Snowflake */ private final SnowflakeTelemetryService telemetryService; @@ -115,6 +116,7 @@ public RecordService() { public void setMetadataConfig(SnowflakeMetadataConfig metadataConfigIn) { metadataConfig = metadataConfigIn; } + /** * extract enableSchematization from the connector config and set the value for the recordService * @@ -316,7 +318,7 @@ public SnowflakeTableRow(SnowflakeRecordContent content, JsonNode metadata) { } void putKey(SinkRecord record, ObjectNode meta) { - if (record.key() == null || record.keySchema() == null) { + if (record.key() == null) { return; } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java index db78eac9b..00a28206a 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/SinkServiceIT.java @@ -579,11 +579,11 @@ public void testNativeNullIngestion() throws Exception { service.closeAll(); } - @Test(expected = SnowflakeKafkaConnectorException.class) + @Test public void testNativeNullValueIngestion() throws Exception { long recordCount = 1; - SinkRecord brokenValue = new SinkRecord(topic, partition, null, null, null, null, 0); + SinkRecord allNullRecord = new SinkRecord(topic, partition, null, null, null, null, 0); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn) @@ -591,7 +591,11 @@ public void testNativeNullValueIngestion() throws Exception { .addTask(table, new TopicPartition(topic, partition)) .build(); - service.insert(brokenValue); + service.insert(allNullRecord); + service.callAllGetOffset(); + TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == recordCount, 30, 20); + + service.closeAll(); } @Test diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 5019ccc78..6937cd722 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -121,7 +121,7 @@ public void afterEach() { } @Test - public void testDefaultTombstoneRecordBehavior() throws Exception { + public void testDefaultTombstoneAndNullRecordBehavior() throws Exception { Map connectorConfig = TestUtils.getConfig(); if (this.ingestionMethod.equals(IngestionMethodConfig.SNOWPIPE)) { From 3fc817f7f707ef0605077d1036be796daad3628c Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 16:37:15 -0700 Subject: [PATCH 39/42] autoformatting --- .../internal/TombstoneRecordIngestionIT.java | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 6937cd722..6853be00f 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -144,22 +144,26 @@ public void testDefaultTombstoneAndNullRecordBehavior() throws Exception { SchemaAndValue input = converter.toConnectData(topic, null); SinkRecord tombstoneRecord1 = new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "tombstoneRecord1", input.schema(), input.value(), 0); + topic, + partition, + Schema.STRING_SCHEMA, + "tombstoneRecord1", + input.schema(), + input.value(), + 0); SinkRecord tombstoneRecord2 = - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, "tombstoneRecord2", null, null, 1); - SinkRecord allNullRecord1 = - new SinkRecord( - topic, partition, null, null, null, null, 2); + new SinkRecord(topic, partition, Schema.STRING_SCHEMA, "tombstoneRecord2", null, null, 1); + SinkRecord allNullRecord1 = new SinkRecord(topic, partition, null, null, null, null, 2); SinkRecord allNullRecord2 = - new SinkRecord( - topic, partition, null, null, input.schema(), input.value(), 3); + new SinkRecord(topic, partition, null, null, input.schema(), input.value(), 3); SinkRecord allNullRecord3 = new SinkRecord( topic, partition, input.schema(), input.value(), input.schema(), input.value(), 4); // test insert - service.insert(Arrays.asList(tombstoneRecord1, tombstoneRecord2, allNullRecord1, allNullRecord2, allNullRecord3)); + service.insert( + Arrays.asList( + tombstoneRecord1, tombstoneRecord2, allNullRecord1, allNullRecord2, allNullRecord3)); service.callAllGetOffset(); // verify inserted From 3fd589b593a460d4e1fc98cbf4032cee9c1bddaa Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 16:58:39 -0700 Subject: [PATCH 40/42] remove null check --- .../connector/records/RecordContentTest.java | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index f8df8b13c..7d04ccb8b 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -280,7 +280,7 @@ public void testColumnNameFormatting() throws JsonProcessingException { @Test public void testGetProcessedRecord() throws JsonProcessingException { SnowflakeJsonConverter jsonConverter = new SnowflakeJsonConverter(); - SchemaAndValue sv = jsonConverter.toConnectData(topic, null); + SchemaAndValue nullSchemaAndValue = jsonConverter.toConnectData(topic, null); String keyStr = "string"; // all null @@ -290,24 +290,13 @@ public void testGetProcessedRecord() throws JsonProcessingException { // null value this.testGetProcessedRecordRunner( new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, sv.schema(), null, partition), + topic, partition, Schema.STRING_SCHEMA, keyStr, nullSchemaAndValue.schema(), null, partition), "{}", keyStr); this.testGetProcessedRecordRunner( - new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, sv.value(), partition), + new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, nullSchemaAndValue.value(), partition), "{}", keyStr); - - // null key - this.testGetProcessedRecordRunner( - new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, null, sv.schema(), sv.value(), partition), - "{}", - ""); - this.testGetProcessedRecordRunner( - new SinkRecord(topic, partition, null, keyStr, sv.schema(), sv.value(), partition), - "{}", - ""); } private void testGetProcessedRecordRunner( From 21cc65a0f2643159de930e8c553b8bbe03641912 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 16:58:42 -0700 Subject: [PATCH 41/42] autoformatting --- .../connector/records/RecordContentTest.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index 7d04ccb8b..cca608cf0 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -290,11 +290,24 @@ public void testGetProcessedRecord() throws JsonProcessingException { // null value this.testGetProcessedRecordRunner( new SinkRecord( - topic, partition, Schema.STRING_SCHEMA, keyStr, nullSchemaAndValue.schema(), null, partition), + topic, + partition, + Schema.STRING_SCHEMA, + keyStr, + nullSchemaAndValue.schema(), + null, + partition), "{}", keyStr); this.testGetProcessedRecordRunner( - new SinkRecord(topic, partition, Schema.STRING_SCHEMA, keyStr, null, nullSchemaAndValue.value(), partition), + new SinkRecord( + topic, + partition, + Schema.STRING_SCHEMA, + keyStr, + null, + nullSchemaAndValue.value(), + partition), "{}", keyStr); } From f516110755ca57f1499606279908e66f36700144 Mon Sep 17 00:00:00 2001 From: revi cheng Date: Thu, 24 Aug 2023 17:11:56 -0700 Subject: [PATCH 42/42] null check schema --- .../connector/records/RecordService.java | 6 ++++ .../connector/records/RecordContentTest.java | 28 +++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java index 6557a8037..610241546 100644 --- a/src/main/java/com/snowflake/kafka/connector/records/RecordService.java +++ b/src/main/java/com/snowflake/kafka/connector/records/RecordService.java @@ -322,6 +322,12 @@ void putKey(SinkRecord record, ObjectNode meta) { return; } + if (record.keySchema() == null) { + throw SnowflakeErrors.ERROR_0010.getException( + "Unsupported Key format, please implement either String Key Converter or Snowflake" + + " Converters"); + } + if (record.keySchema().toString().equals(Schema.STRING_SCHEMA.toString())) { meta.put(KEY, record.key().toString()); } else if (SnowflakeJsonSchema.NAME.equals(record.keySchema().name())) { diff --git a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java index cca608cf0..769dc8c19 100644 --- a/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java +++ b/src/test/java/com/snowflake/kafka/connector/records/RecordContentTest.java @@ -310,6 +310,34 @@ public void testGetProcessedRecord() throws JsonProcessingException { partition), "{}", keyStr); + + // null key + this.testGetProcessedRecordRunner( + new SinkRecord( + topic, + partition, + Schema.STRING_SCHEMA, + null, + nullSchemaAndValue.schema(), + nullSchemaAndValue.value(), + partition), + "{}", + ""); + try { + this.testGetProcessedRecordRunner( + new SinkRecord( + topic, + partition, + null, + keyStr, + nullSchemaAndValue.schema(), + nullSchemaAndValue.value(), + partition), + "{}", + keyStr); + } catch (SnowflakeKafkaConnectorException ex) { + assert ex.checkErrorCode(SnowflakeErrors.ERROR_0010); + } } private void testGetProcessedRecordRunner(